cross merge fullstack_release_candidate into trunk

git-svn-id: https://hyracks.googlecode.com/svn/trunk/fullstack@3208 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks/hyracks-api/pom.xml b/hyracks/hyracks-api/pom.xml
index 72f0d8b..6807f76 100644
--- a/hyracks/hyracks-api/pom.xml
+++ b/hyracks/hyracks-api/pom.xml
@@ -15,8 +15,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
index a8f2fda..a4f0b29 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface IInputChannel {
@@ -30,7 +30,7 @@
 
     public void recycleBuffer(ByteBuffer buffer);
 
-    public void open(IHyracksTaskContext ctx) throws HyracksDataException;
+    public void open(IHyracksCommonContext ctx) throws HyracksDataException;
 
     public void close() throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
index e34e60d..cd2b698 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -17,6 +17,8 @@
 import java.io.Serializable;
 import java.util.EnumSet;
 
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 
@@ -30,6 +32,10 @@
         CREATE_JOB,
         GET_JOB_STATUS,
         START_JOB,
+        GET_DATASET_DIRECTORY_SERIVICE_INFO,
+        GET_DATASET_RESULT_STATUS,
+        GET_DATASET_RECORD_DESCRIPTOR,
+        GET_DATASET_RESULT_LOCATIONS,
         WAIT_FOR_COMPLETION,
         GET_NODE_CONTROLLERS_INFO
     }
@@ -156,6 +162,74 @@
         }
     }
 
+    public static class GetDatasetDirectoryServiceInfoFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.GET_DATASET_DIRECTORY_SERIVICE_INFO;
+        }
+    }
+
+    public static class GetDatasetResultStatusFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+
+        private final ResultSetId rsId;
+
+        public GetDatasetResultStatusFunction(JobId jobId, ResultSetId rsId) {
+            this.jobId = jobId;
+            this.rsId = rsId;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.GET_DATASET_RESULT_STATUS;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public ResultSetId getResultSetId() {
+            return rsId;
+        }
+    }
+
+    public static class GetDatasetResultLocationsFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+
+        private final ResultSetId rsId;
+
+        private final DatasetDirectoryRecord[] knownRecords;
+
+        public GetDatasetResultLocationsFunction(JobId jobId, ResultSetId rsId, DatasetDirectoryRecord[] knownRecords) {
+            this.jobId = jobId;
+            this.rsId = rsId;
+            this.knownRecords = knownRecords;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.GET_DATASET_RESULT_LOCATIONS;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public ResultSetId getResultSetId() {
+            return rsId;
+        }
+
+        public DatasetDirectoryRecord[] getKnownRecords() {
+            return knownRecords;
+        }
+    }
+
     public static class WaitForCompletionFunction extends Function {
         private static final long serialVersionUID = 1L;
 
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
index 4c06d42..2ab42c0 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
@@ -17,6 +17,7 @@
 import java.util.EnumSet;
 import java.util.Map;
 
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -76,6 +77,12 @@
     }
 
     @Override
+    public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception {
+        HyracksClientInterfaceFunctions.GetDatasetDirectoryServiceInfoFunction gddsf = new HyracksClientInterfaceFunctions.GetDatasetDirectoryServiceInfoFunction();
+        return (NetworkAddress) rpci.call(ipcHandle, gddsf);
+    }
+
+    @Override
     public void waitForCompletion(JobId jobId) throws Exception {
         HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = new HyracksClientInterfaceFunctions.WaitForCompletionFunction(
                 jobId);
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
index 227524c..e0fafb0 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
@@ -26,6 +26,7 @@
 import org.apache.http.impl.client.DefaultHttpClient;
 
 import edu.uci.ics.hyracks.api.client.impl.JobSpecificationActivityClusterGraphGeneratorFactory;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
 import edu.uci.ics.hyracks.api.job.JobFlag;
@@ -118,6 +119,10 @@
         return hci.startJob(appName, JavaSerializationUtils.serialize(acggf), jobFlags);
     }
 
+    public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception {
+        return hci.getDatasetDirectoryServiceInfo();
+    }
+
     @Override
     public void waitForCompletion(JobId jobId) throws Exception {
         hci.waitForCompletion(jobId);
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
index bdbb544..6333c22 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
@@ -18,6 +18,7 @@
 import java.util.EnumSet;
 import java.util.Map;
 
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
@@ -100,6 +101,14 @@
             throws Exception;
 
     /**
+     * Gets the IP Address and port for the DatasetDirectoryService wrapped in NetworkAddress
+     * 
+     * @return {@link NetworkAddress}
+     * @throws Exception
+     */
+    public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
+
+    /**
      * Waits until the specified job has completed, either successfully or has
      * encountered a permanent failure.
      * 
@@ -123,4 +132,4 @@
      * @throws Exception
      */
     public ClusterTopology getClusterTopology() throws Exception;
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
index ef5906e..22b0a8f 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
@@ -17,6 +17,7 @@
 import java.util.EnumSet;
 import java.util.Map;
 
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -35,6 +36,8 @@
 
     public JobId startJob(String appName, byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception;
 
+    public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
+
     public void waitForCompletion(JobId jobId) throws Exception;
 
     public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception;
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
index fd9218a..73b5488 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
@@ -27,10 +27,14 @@
 
     private final NetworkAddress netAddress;
 
-    public NodeControllerInfo(String nodeId, NodeStatus status, NetworkAddress netAddress) {
+    private final NetworkAddress datasetNetworkAddress;
+
+    public NodeControllerInfo(String nodeId, NodeStatus status, NetworkAddress netAddress,
+            NetworkAddress datasetNetworkAddress) {
         this.nodeId = nodeId;
         this.status = status;
         this.netAddress = netAddress;
+        this.datasetNetworkAddress = datasetNetworkAddress;
     }
 
     public String getNodeId() {
@@ -44,4 +48,8 @@
     public NetworkAddress getNetworkAddress() {
         return netAddress;
     }
+
+    public NetworkAddress getDatasetNetworkAddress() {
+        return datasetNetworkAddress;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
index f36b7b3..0eac9a2 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
@@ -17,6 +17,7 @@
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.rewriter.ActivityClusterGraphRewriter;
 
 public class JobSpecificationActivityClusterGraphGeneratorFactory implements IActivityClusterGraphGeneratorFactory {
     private static final long serialVersionUID = 1L;
@@ -78,6 +79,8 @@
         return new IActivityClusterGraphGenerator() {
             @Override
             public ActivityClusterGraph initialize() {
+                ActivityClusterGraphRewriter rewriter = new ActivityClusterGraphRewriter();
+                rewriter.rewrite(acg);
                 return acg;
             }
 
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
index e964d66..a2ee977 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
@@ -15,6 +15,7 @@
 package edu.uci.ics.hyracks.api.context;
 
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
 import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
@@ -28,5 +29,7 @@
 
     public ICounterContext getCounterContext();
 
+    public IDatasetPartitionManager getDatasetPartitionManager();
+
     public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializer.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializer.java
new file mode 100644
index 0000000..ba2ff9a
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializer.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.api.dataflow.value;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IResultSerializer extends Serializable {
+    /**
+     * Initializes the serializer.
+     */
+    public void init() throws HyracksDataException;
+
+    /**
+     * Method to serialize the result and append it to the provided output stream
+     * 
+     * @param tAccess
+     *            - A frame tuple accessor object that contains the original data to be serialized
+     * @param tIdx
+     *            - Index of the tuple that should be serialized.
+     * @return true if the tuple was appended successfully, else false.
+     */
+    public boolean appendTuple(IFrameTupleAccessor tAccess, int tIdx) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializerFactory.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializerFactory.java
new file mode 100644
index 0000000..1fbf00f
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializerFactory.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.api.dataflow.value;
+
+import java.io.PrintStream;
+import java.io.Serializable;
+
+public interface IResultSerializerFactory extends Serializable {
+    /**
+     * Creates a result serialized appender
+     * 
+     * @param printStream
+     *            - A print stream object to which the serialized results will be written.
+     * @return A new instance of result serialized appender.
+     */
+    public IResultSerializer createResultSerializer(RecordDescriptor recordDesc, PrintStream printStream);
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/JSONSerializable.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/JSONSerializable.java
new file mode 100644
index 0000000..1eca502
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/JSONSerializable.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.api.dataflow.value;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+public interface JSONSerializable {
+    /**
+     * Returns the JSON representation of the object.
+     * 
+     * @return A new JSONObject instance representing this Java object.
+     */
+    public JSONObject toJSON() throws JSONException;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/DatasetDirectoryRecord.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/DatasetDirectoryRecord.java
new file mode 100644
index 0000000..6316bba
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/DatasetDirectoryRecord.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.api.dataset;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+
+public class DatasetDirectoryRecord implements Serializable {
+    public enum Status {
+        IDLE,
+        RUNNING,
+        SUCCESS,
+        FAILED
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    private NetworkAddress address;
+
+    private boolean readEOS;
+
+    private Status status;
+
+    public DatasetDirectoryRecord() {
+        this.address = null;
+        this.readEOS = false;
+        this.status = Status.IDLE;
+    }
+
+    public void setNetworkAddress(NetworkAddress address) {
+        this.address = address;
+    }
+
+    public NetworkAddress getNetworkAddress() {
+        return address;
+    }
+
+    public void readEOS() {
+        this.readEOS = true;
+    }
+
+    public boolean hasReachedReadEOS() {
+        return readEOS;
+    }
+
+    public void start() {
+        status = Status.RUNNING;
+    }
+
+    public void writeEOS() {
+        status = Status.SUCCESS;
+    }
+
+    public void fail() {
+        status = Status.FAILED;
+    }
+
+    public Status getStatus() {
+        return status;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof DatasetDirectoryRecord)) {
+            return false;
+        }
+        return address.equals(((DatasetDirectoryRecord) o).address);
+    }
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetDirectoryService.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetDirectoryService.java
new file mode 100644
index 0000000..5266333
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetDirectoryService.java
@@ -0,0 +1,34 @@
+/*
+ * 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.api.dataset;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IDatasetDirectoryService {
+    public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
+            int nPartitions, NetworkAddress networkAddress);
+
+    public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition);
+
+    public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition);
+
+    public Status getResultStatus(JobId jobId, ResultSetId rsId) throws HyracksDataException;
+
+    public DatasetDirectoryRecord[] getResultPartitionLocations(JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownLocations) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetInputChannelMonitor.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetInputChannelMonitor.java
new file mode 100644
index 0000000..65ba1c7
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetInputChannelMonitor.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.api.dataset;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+
+public interface IDatasetInputChannelMonitor extends IInputChannelMonitor {
+    public boolean eosReached();
+
+    public boolean failed();
+
+    public int getNFramesAvailable();
+
+    public void notifyFrameRead();
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java
new file mode 100644
index 0000000..ae38c7f
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java
@@ -0,0 +1,37 @@
+/*
+ * 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.api.dataset;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IDatasetPartitionManager {
+    public IFrameWriter createDatasetPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
+            int partition, int nPartitions) throws HyracksException;
+
+    public void reportPartitionWriteCompletion(JobId jobId, ResultSetId resultSetId, int partition)
+            throws HyracksException;
+
+    public void reportPartitionFailure(JobId jobId, ResultSetId resultSetId, int partition) throws HyracksException;
+
+    public void initializeDatasetPartitionReader(JobId jobId, int partition, IFrameWriter noc) throws HyracksException;
+
+    public IWorkspaceFileFactory getFileFactory();
+
+    public void close();
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java
new file mode 100644
index 0000000..8f5ed64
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.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.api.dataset;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+
+public interface IDatasetPartitionReader {
+    public void writeTo(IFrameWriter writer);
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionWriter.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionWriter.java
new file mode 100644
index 0000000..42dc157
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionWriter.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.api.dataset;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IDatasetPartitionWriter extends IFrameWriter {
+    public Page returnPage() throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDataset.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDataset.java
new file mode 100644
index 0000000..4a7a6b0
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDataset.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.api.dataset;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IHyracksDataset {
+    public IHyracksDatasetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java
new file mode 100644
index 0000000..d49d5cd
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.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.api.dataset;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IHyracksDatasetDirectoryServiceConnection {
+    /**
+     * Gets the result status for the given result set.
+     * 
+     * @param jobId
+     *            ID of the job
+     * @param rsId
+     *            ID of the result set
+     * @return {@link Status}
+     * @throws Exception
+     */
+    public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception;
+
+    /**
+     * Gets the IP Addresses and ports for the partition generating the result for each location.
+     * 
+     * @param jobId
+     *            ID of the job
+     * @param rsId
+     *            ID of the result set
+     * @param knownRecords
+     *            Locations that are already known to the client
+     * @return {@link NetworkAddress[]}
+     * @throws Exception
+     */
+    public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownRecords) throws Exception;
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java
new file mode 100644
index 0000000..ba21a84
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.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.api.dataset;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IHyracksDatasetDirectoryServiceInterface {
+    /**
+     * Gets the result status for the given result set.
+     * 
+     * @param jobId
+     *            ID of the job
+     * @param rsId
+     *            ID of the result set
+     * @return {@link Status}
+     * @throws Exception
+     */
+    public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception;
+
+    /**
+     * Gets the IP Addresses and ports for the partition generating the result for each location.
+     * 
+     * @param jobId
+     *            ID of the job
+     * @param rsId
+     *            ID of the result set
+     * @param knownRecords
+     *            Locations from the dataset directory that are already known to the client
+     * @return {@link NetworkAddress[]}
+     * @throws Exception
+     */
+    public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownRecords) throws Exception;
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
new file mode 100644
index 0000000..b928a49
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
@@ -0,0 +1,26 @@
+/*
+ * 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.api.dataset;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IHyracksDatasetReader {
+    public Status getResultStatus();
+
+    public int read(ByteBuffer buffer) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/Page.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/Page.java
new file mode 100644
index 0000000..7275dfd
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/Page.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.api.dataset;
+
+import java.nio.ByteBuffer;
+
+public class Page {
+    private final ByteBuffer buffer;
+
+    public Page(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    public ByteBuffer clear() {
+        return (ByteBuffer) buffer.clear();
+    }
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/ResultSetId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/ResultSetId.java
new file mode 100644
index 0000000..ae38ef3
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/ResultSetId.java
@@ -0,0 +1,52 @@
+/*
+ * 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.api.dataset;
+
+import java.io.Serializable;
+
+public class ResultSetId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final long id;
+
+    public ResultSetId(long id) {
+        this.id = id;
+    }
+
+    public long getId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int) id;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof ResultSetId)) {
+            return false;
+        }
+        return ((ResultSetId) o).id == id;
+    }
+
+    @Override
+    public String toString() {
+        return "RSID:" + id;
+    }
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/ActivityCluster.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/ActivityCluster.java
index 6698ff7..9fb2b08 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/ActivityCluster.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/ActivityCluster.java
@@ -33,7 +33,7 @@
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 
-public final class ActivityCluster implements Serializable {
+public class ActivityCluster implements Serializable {
     private static final long serialVersionUID = 1L;
 
     private final ActivityClusterGraph acg;
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
index 7c523f1..1fdff0f 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
@@ -34,12 +34,15 @@
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 
 public class JobSpecification implements Serializable, IOperatorDescriptorRegistry, IConnectorDescriptorRegistry {
     private static final long serialVersionUID = 1L;
 
     private final List<OperatorDescriptorId> roots;
 
+    private final List<ResultSetId> resultSetIds;
+
     private final Map<OperatorDescriptorId, IOperatorDescriptor> opMap;
 
     private final Map<ConnectorDescriptorId, IConnectorDescriptor> connMap;
@@ -72,6 +75,7 @@
 
     public JobSpecification() {
         roots = new ArrayList<OperatorDescriptorId>();
+        resultSetIds = new ArrayList<ResultSetId>();
         opMap = new HashMap<OperatorDescriptorId, IOperatorDescriptor>();
         connMap = new HashMap<ConnectorDescriptorId, IConnectorDescriptor>();
         opInputMap = new HashMap<OperatorDescriptorId, List<IConnectorDescriptor>>();
@@ -104,6 +108,10 @@
         roots.add(op.getOperatorId());
     }
 
+    public void addResultSetId(ResultSetId rsId) {
+        resultSetIds.add(rsId);
+    }
+
     public void connect(IConnectorDescriptor conn, IOperatorDescriptor producerOp, int producerPort,
             IOperatorDescriptor consumerOp, int consumerPort) {
         insertIntoIndexedMap(opInputMap, consumerOp.getOperatorId(), consumerPort, conn);
@@ -208,6 +216,10 @@
         return roots;
     }
 
+    public List<ResultSetId> getResultSetIds() {
+        return resultSetIds;
+    }
+
     public IConnectorPolicyAssignmentPolicy getConnectorPolicyAssignmentPolicy() {
         return connectorPolicyAssignmentPolicy;
     }
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/ResultSetPartitionId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/ResultSetPartitionId.java
new file mode 100644
index 0000000..148a8a2
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/ResultSetPartitionId.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.api.partitions;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public final class ResultSetPartitionId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final JobId jobId;
+
+    private final ResultSetId resultSetId;
+    
+    private final int partition;
+
+    public ResultSetPartitionId(JobId jobId, ResultSetId resultSetId, int partition) {
+        this.jobId = jobId;
+        this.resultSetId = resultSetId;
+        this.partition = partition;
+    }
+
+    public JobId getJobId() {
+        return jobId;
+    }
+
+    public ResultSetId getResultSetId() {
+        return resultSetId;
+    }
+
+    public int getPartition() {
+        return partition;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((resultSetId == null) ? 0 : resultSetId.hashCode());
+        result = prime * result + ((jobId == null) ? 0 : jobId.hashCode());
+        result = prime * result + partition;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        ResultSetPartitionId other = (ResultSetPartitionId) obj;
+        if (resultSetId == null) {
+            if (other.resultSetId != null)
+                return false;
+        } else if (!resultSetId.equals(other.resultSetId))
+            return false;
+        if (jobId == null) {
+            if (other.jobId != null)
+                return false;
+        } else if (!jobId.equals(other.jobId))
+            return false;
+        if (partition != other.partition)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return jobId.toString() + ":" + resultSetId + ":" + partition;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/ActivityClusterGraphRewriter.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/ActivityClusterGraphRewriter.java
new file mode 100644
index 0000000..c6761e9
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/ActivityClusterGraphRewriter.java
@@ -0,0 +1,381 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.rewriter;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Queue;
+import java.util.Set;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.ActivityCluster;
+import edu.uci.ics.hyracks.api.job.ActivityClusterGraph;
+import edu.uci.ics.hyracks.api.job.ActivityClusterId;
+import edu.uci.ics.hyracks.api.rewriter.runtime.SuperActivity;
+
+/**
+ * This class rewrite the AcivityClusterGraph to eliminate
+ * all one-to-one connections and merge one-to-one connected
+ * DAGs into super activities.
+ * </p>
+ * Each super activity internally maintains a DAG and execute it at the runtime.
+ * 
+ * @author yingyib
+ */
+public class ActivityClusterGraphRewriter {
+    private static String ONE_TO_ONE_CONNECTOR = "edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor";
+
+    /**
+     * rewrite an activity cluster graph to eliminate
+     * all one-to-one connections and merge one-to-one connected
+     * DAGs into super activities.
+     * 
+     * @param acg
+     *            the activity cluster graph
+     */
+    public void rewrite(ActivityClusterGraph acg) {
+        acg.getActivityMap().clear();
+        acg.getConnectorMap().clear();
+        Map<IActivity, SuperActivity> invertedActivitySuperActivityMap = new HashMap<IActivity, SuperActivity>();
+        for (Entry<ActivityClusterId, ActivityCluster> entry : acg.getActivityClusterMap().entrySet()) {
+            rewriteIntraActivityCluster(entry.getValue(), invertedActivitySuperActivityMap);
+        }
+        for (Entry<ActivityClusterId, ActivityCluster> entry : acg.getActivityClusterMap().entrySet()) {
+            rewriteInterActivityCluster(entry.getValue(), invertedActivitySuperActivityMap);
+        }
+        invertedActivitySuperActivityMap.clear();
+    }
+
+    /**
+     * rewrite the blocking relationship among activity cluster
+     * 
+     * @param ac
+     *            the activity cluster to be rewritten
+     */
+    private void rewriteInterActivityCluster(ActivityCluster ac,
+            Map<IActivity, SuperActivity> invertedActivitySuperActivityMap) {
+        Map<ActivityId, Set<ActivityId>> blocked2BlockerMap = ac.getBlocked2BlockerMap();
+        Map<ActivityId, ActivityId> invertedAid2SuperAidMap = new HashMap<ActivityId, ActivityId>();
+        for (Entry<IActivity, SuperActivity> entry : invertedActivitySuperActivityMap.entrySet()) {
+            invertedAid2SuperAidMap.put(entry.getKey().getActivityId(), entry.getValue().getActivityId());
+        }
+        Map<ActivityId, Set<ActivityId>> replacedBlocked2BlockerMap = new HashMap<ActivityId, Set<ActivityId>>();
+        for (Entry<ActivityId, Set<ActivityId>> entry : blocked2BlockerMap.entrySet()) {
+            ActivityId blocked = entry.getKey();
+            ActivityId replacedBlocked = invertedAid2SuperAidMap.get(blocked);
+            Set<ActivityId> blockers = entry.getValue();
+            Set<ActivityId> replacedBlockers = null;
+            if (blockers != null) {
+                replacedBlockers = new HashSet<ActivityId>();
+                for (ActivityId blocker : blockers) {
+                    replacedBlockers.add(invertedAid2SuperAidMap.get(blocker));
+                    ActivityCluster dependingAc = ac.getActivityClusterGraph().getActivityMap()
+                            .get(invertedAid2SuperAidMap.get(blocker));
+                    if (!ac.getDependencies().contains(dependingAc)) {
+                        ac.getDependencies().add(dependingAc);
+                    }
+                }
+            }
+            if (replacedBlockers != null) {
+                Set<ActivityId> existingBlockers = replacedBlocked2BlockerMap.get(replacedBlocked);
+                if (existingBlockers == null) {
+                    replacedBlocked2BlockerMap.put(replacedBlocked, replacedBlockers);
+                } else {
+                    existingBlockers.addAll(replacedBlockers);
+                    replacedBlocked2BlockerMap.put(replacedBlocked, existingBlockers);
+                }
+            }
+        }
+        blocked2BlockerMap.clear();
+        blocked2BlockerMap.putAll(replacedBlocked2BlockerMap);
+    }
+
+    /**
+     * rewrite an activity cluster internally
+     * 
+     * @param ac
+     *            the activity cluster to be rewritten
+     */
+    private void rewriteIntraActivityCluster(ActivityCluster ac,
+            Map<IActivity, SuperActivity> invertedActivitySuperActivityMap) {
+        Map<ActivityId, IActivity> activities = ac.getActivityMap();
+        Map<ActivityId, List<IConnectorDescriptor>> activityInputMap = ac.getActivityInputMap();
+        Map<ActivityId, List<IConnectorDescriptor>> activityOutputMap = ac.getActivityOutputMap();
+        Map<ConnectorDescriptorId, Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>> connectorActivityMap = ac
+                .getConnectorActivityMap();
+        ActivityClusterGraph acg = ac.getActivityClusterGraph();
+        Map<ActivityId, IActivity> startActivities = new HashMap<ActivityId, IActivity>();
+        Map<ActivityId, SuperActivity> superActivities = new HashMap<ActivityId, SuperActivity>();
+        Map<ActivityId, Queue<IActivity>> toBeExpendedMap = new HashMap<ActivityId, Queue<IActivity>>();
+
+        /**
+         * Build the initial super activities
+         */
+        for (Entry<ActivityId, IActivity> entry : activities.entrySet()) {
+            ActivityId activityId = entry.getKey();
+            IActivity activity = entry.getValue();
+            if (activityInputMap.get(activityId) == null) {
+                startActivities.put(activityId, activity);
+                /**
+                 * use the start activity's id as the id of the super activity
+                 */
+                createNewSuperActivity(ac, superActivities, toBeExpendedMap, invertedActivitySuperActivityMap,
+                        activityId, activity);
+            }
+        }
+
+        /**
+         * expand one-to-one connected activity cluster by the BFS order.
+         * after the while-loop, the original activities are partitioned
+         * into equivalent classes, one-per-super-activity.
+         */
+        Map<ActivityId, SuperActivity> clonedSuperActivities = new HashMap<ActivityId, SuperActivity>();
+        while (toBeExpendedMap.size() > 0) {
+            clonedSuperActivities.clear();
+            clonedSuperActivities.putAll(superActivities);
+            for (Entry<ActivityId, SuperActivity> entry : clonedSuperActivities.entrySet()) {
+                ActivityId superActivityId = entry.getKey();
+                SuperActivity superActivity = entry.getValue();
+
+                /**
+                 * for the case where the super activity has already been swallowed
+                 */
+                if (superActivities.get(superActivityId) == null) {
+                    continue;
+                }
+
+                /**
+                 * expend the super activity
+                 */
+                Queue<IActivity> toBeExpended = toBeExpendedMap.get(superActivityId);
+                if (toBeExpended == null) {
+                    /**
+                     * Nothing to expand
+                     */
+                    continue;
+                }
+                IActivity expendingActivity = toBeExpended.poll();
+                List<IConnectorDescriptor> outputConnectors = activityOutputMap.get(expendingActivity.getActivityId());
+                if (outputConnectors != null) {
+                    for (IConnectorDescriptor outputConn : outputConnectors) {
+                        Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>> endPoints = connectorActivityMap
+                                .get(outputConn.getConnectorId());
+                        IActivity newActivity = endPoints.getRight().getLeft();
+                        SuperActivity existingSuperActivity = invertedActivitySuperActivityMap.get(newActivity);
+                        if (outputConn.getClass().getName().contains(ONE_TO_ONE_CONNECTOR)) {
+                            /**
+                             * expend the super activity cluster on an one-to-one out-bound connection
+                             */
+                            if (existingSuperActivity == null) {
+                                superActivity.addActivity(newActivity);
+                                toBeExpended.add(newActivity);
+                                invertedActivitySuperActivityMap.put(newActivity, superActivity);
+                            } else {
+                                /**
+                                 * the two activities already in the same super activity
+                                 */
+                                if (existingSuperActivity == superActivity) {
+                                    continue;
+                                }
+                                /**
+                                 * swallow an existing super activity
+                                 */
+                                swallowExistingSuperActivity(superActivities, toBeExpendedMap,
+                                        invertedActivitySuperActivityMap, superActivity, superActivityId,
+                                        existingSuperActivity);
+                            }
+                        } else {
+                            if (existingSuperActivity == null) {
+                                /**
+                                 * create new activity
+                                 */
+                                createNewSuperActivity(ac, superActivities, toBeExpendedMap,
+                                        invertedActivitySuperActivityMap, newActivity.getActivityId(), newActivity);
+                            }
+                        }
+                    }
+                }
+
+                /**
+                 * remove the to-be-expended queue if it is empty
+                 */
+                if (toBeExpended.size() == 0) {
+                    toBeExpendedMap.remove(superActivityId);
+                }
+            }
+        }
+
+        Map<ConnectorDescriptorId, IConnectorDescriptor> connMap = ac.getConnectorMap();
+        Map<ConnectorDescriptorId, RecordDescriptor> connRecordDesc = ac.getConnectorRecordDescriptorMap();
+        Map<SuperActivity, Integer> superActivityProducerPort = new HashMap<SuperActivity, Integer>();
+        Map<SuperActivity, Integer> superActivityConsumerPort = new HashMap<SuperActivity, Integer>();
+        for (Entry<ActivityId, SuperActivity> entry : superActivities.entrySet()) {
+            superActivityProducerPort.put(entry.getValue(), 0);
+            superActivityConsumerPort.put(entry.getValue(), 0);
+        }
+
+        /**
+         * create a new activity cluster to replace the old activity cluster
+         */
+        ActivityCluster newActivityCluster = new ActivityCluster(acg, ac.getId());
+        newActivityCluster.setConnectorPolicyAssignmentPolicy(ac.getConnectorPolicyAssignmentPolicy());
+        for (Entry<ActivityId, SuperActivity> entry : superActivities.entrySet()) {
+            newActivityCluster.addActivity(entry.getValue());
+            acg.getActivityMap().put(entry.getKey(), newActivityCluster);
+        }
+
+        /**
+         * Setup connectors: either inside a super activity or among super activities
+         */
+        for (Entry<ConnectorDescriptorId, Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>> entry : connectorActivityMap
+                .entrySet()) {
+            ConnectorDescriptorId connectorId = entry.getKey();
+            Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>> endPoints = entry.getValue();
+            IActivity producerActivity = endPoints.getLeft().getLeft();
+            IActivity consumerActivity = endPoints.getRight().getLeft();
+            int producerPort = endPoints.getLeft().getRight();
+            int consumerPort = endPoints.getRight().getRight();
+            RecordDescriptor recordDescriptor = connRecordDesc.get(connectorId);
+            IConnectorDescriptor conn = connMap.get(connectorId);
+            if (conn.getClass().getName().contains(ONE_TO_ONE_CONNECTOR)) {
+                /**
+                 * connection edge between inner activities
+                 */
+                SuperActivity residingSuperActivity = invertedActivitySuperActivityMap.get(producerActivity);
+                residingSuperActivity.connect(conn, producerActivity, producerPort, consumerActivity, consumerPort,
+                        recordDescriptor);
+            } else {
+                /**
+                 * connection edge between super activities
+                 */
+                SuperActivity producerSuperActivity = invertedActivitySuperActivityMap.get(producerActivity);
+                SuperActivity consumerSuperActivity = invertedActivitySuperActivityMap.get(consumerActivity);
+                int producerSAPort = superActivityProducerPort.get(producerSuperActivity);
+                int consumerSAPort = superActivityConsumerPort.get(consumerSuperActivity);
+                newActivityCluster.addConnector(conn);
+                newActivityCluster.connect(conn, producerSuperActivity, producerSAPort, consumerSuperActivity,
+                        consumerSAPort, recordDescriptor);
+
+                /**
+                 * bridge the port
+                 */
+                producerSuperActivity.setClusterOutputIndex(producerSAPort, producerActivity.getActivityId(),
+                        producerPort);
+                consumerSuperActivity.setClusterInputIndex(consumerSAPort, consumerActivity.getActivityId(),
+                        consumerPort);
+                acg.getConnectorMap().put(connectorId, newActivityCluster);
+
+                /**
+                 * increasing the port number for the producer and consumer
+                 */
+                superActivityProducerPort.put(producerSuperActivity, ++producerSAPort);
+                superActivityConsumerPort.put(consumerSuperActivity, ++consumerSAPort);
+            }
+        }
+
+        /**
+         * Set up the roots of the new activity cluster
+         */
+        for (Entry<ActivityId, SuperActivity> entry : superActivities.entrySet()) {
+            List<IConnectorDescriptor> connIds = newActivityCluster.getActivityOutputMap().get(entry.getKey());
+            if (connIds == null || connIds.size() == 0) {
+                newActivityCluster.addRoot(entry.getValue());
+            }
+        }
+
+        /**
+         * set up the blocked2Blocker mapping, which will be updated in the rewriteInterActivityCluster call
+         */
+        newActivityCluster.getBlocked2BlockerMap().putAll(ac.getBlocked2BlockerMap());
+
+        /**
+         * replace the old activity cluster with the new activity cluster
+         */
+        acg.getActivityClusterMap().put(ac.getId(), newActivityCluster);
+    }
+
+    /**
+     * Create a new super activity
+     * 
+     * @param acg
+     *            the activity cluster
+     * @param superActivities
+     *            the map from activity id to current super activities
+     * @param toBeExpendedMap
+     *            the map from an existing super activity to its BFS expansion queue of the original activities
+     * @param invertedActivitySuperActivityMap
+     *            the map from the original activities to their hosted super activities
+     * @param activityId
+     *            the activity id for the new super activity, which is the first added acitivty's id in the super activity
+     * @param activity
+     *            the first activity added to the new super activity
+     */
+    private void createNewSuperActivity(ActivityCluster acg, Map<ActivityId, SuperActivity> superActivities,
+            Map<ActivityId, Queue<IActivity>> toBeExpendedMap,
+            Map<IActivity, SuperActivity> invertedActivitySuperActivityMap, ActivityId activityId, IActivity activity) {
+        SuperActivity superActivity = new SuperActivity(acg.getActivityClusterGraph(), acg.getId(), activityId);
+        superActivities.put(activityId, superActivity);
+        superActivity.addActivity(activity);
+        Queue<IActivity> toBeExpended = new LinkedList<IActivity>();
+        toBeExpended.add(activity);
+        toBeExpendedMap.put(activityId, toBeExpended);
+        invertedActivitySuperActivityMap.put(activity, superActivity);
+    }
+
+    /**
+     * One super activity swallows another existing super activity.
+     * 
+     * @param superActivities
+     *            the map from activity id to current super activities
+     * @param toBeExpendedMap
+     *            the map from an existing super activity to its BFS expansion queue of the original activities
+     * @param invertedActivitySuperActivityMap
+     *            the map from the original activities to their hosted super activities
+     * @param superActivity
+     *            the "swallowing" super activity
+     * @param superActivityId
+     *            the activity id for the "swallowing" super activity, which is also the first added acitivty's id in the super activity
+     * @param existingSuperActivity
+     *            an existing super activity which is to be swallowed by the "swallowing" super activity
+     */
+    private void swallowExistingSuperActivity(Map<ActivityId, SuperActivity> superActivities,
+            Map<ActivityId, Queue<IActivity>> toBeExpendedMap,
+            Map<IActivity, SuperActivity> invertedActivitySuperActivityMap, SuperActivity superActivity,
+            ActivityId superActivityId, SuperActivity existingSuperActivity) {
+        ActivityId existingSuperActivityId = existingSuperActivity.getActivityId();
+        superActivities.remove(existingSuperActivityId);
+        for (Entry<ActivityId, IActivity> existingEntry : existingSuperActivity.getActivityMap().entrySet()) {
+            IActivity existingActivity = existingEntry.getValue();
+            superActivity.addActivity(existingActivity);
+            invertedActivitySuperActivityMap.put(existingActivity, superActivity);
+        }
+        Queue<IActivity> tbeQueue = toBeExpendedMap.get(superActivityId);
+        Queue<IActivity> existingTbeQueque = toBeExpendedMap.remove(existingSuperActivityId);
+        if (existingTbeQueque != null) {
+            tbeQueue.addAll(existingTbeQueque);
+        }
+    }
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/OneToOneConnectedActivityCluster.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/OneToOneConnectedActivityCluster.java
new file mode 100644
index 0000000..07b7ffc
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/OneToOneConnectedActivityCluster.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.rewriter;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.job.ActivityCluster;
+import edu.uci.ics.hyracks.api.job.ActivityClusterGraph;
+import edu.uci.ics.hyracks.api.job.ActivityClusterId;
+
+/**
+ * All the connectors in an OneToOneConnectedCluster are OneToOneConnectorDescriptors.
+ * 
+ * @author yingyib
+ */
+public class OneToOneConnectedActivityCluster extends ActivityCluster {
+
+    private static final long serialVersionUID = 1L;
+
+    protected final Map<Integer, Pair<ActivityId, Integer>> clusterInputIndexMap = new HashMap<Integer, Pair<ActivityId, Integer>>();
+    protected final Map<Integer, Pair<ActivityId, Integer>> clusterOutputIndexMap = new HashMap<Integer, Pair<ActivityId, Integer>>();
+    protected final Map<Pair<ActivityId, Integer>, Integer> invertedClusterOutputIndexMap = new HashMap<Pair<ActivityId, Integer>, Integer>();
+    protected final Map<Pair<ActivityId, Integer>, Integer> invertedClusterInputIndexMap = new HashMap<Pair<ActivityId, Integer>, Integer>();
+
+    public OneToOneConnectedActivityCluster(ActivityClusterGraph acg, ActivityClusterId id) {
+        super(acg, id);
+    }
+
+    /**
+     * Set up the mapping of the cluster's output channel to an internal activity and its output channel
+     * 
+     * @param clusterOutputIndex
+     *            the output channel index for the cluster
+     * @param activityId
+     *            the id of the internal activity which produces the corresponding output
+     * @param activityOutputIndex
+     *            the output channel index of the internal activity which corresponds to the output channel of the cluster of activities
+     */
+    public void setClusterOutputIndex(int clusterOutputIndex, ActivityId activityId, int activityOutputIndex) {
+        clusterOutputIndexMap.put(clusterOutputIndex, Pair.of(activityId, activityOutputIndex));
+        invertedClusterOutputIndexMap.put(Pair.of(activityId, activityOutputIndex), clusterOutputIndex);
+    }
+
+    /**
+     * get the an internal activity and its output channel of a cluster output channel
+     * 
+     * @param clusterOutputIndex
+     *            the output channel index for the cluster
+     * @return a pair containing the activity id of the corresponding internal activity and the output channel index
+     */
+    public Pair<ActivityId, Integer> getActivityIdOutputIndex(int clusterOutputIndex) {
+        return clusterOutputIndexMap.get(clusterOutputIndex);
+    }
+
+    /**
+     * Set up the mapping of the cluster's input channel to an internal activity and input output channel
+     * 
+     * @param clusterInputIndex
+     *            the input channel index for the cluster
+     * @param activityId
+     *            the id of the internal activity which consumes the corresponding input
+     * @param activityInputIndex
+     *            the output channel index of the internal activity which corresponds to the input channel of the cluster of activities
+     */
+    public void setClusterInputIndex(int clusterInputIndex, ActivityId activityId, int activityInputIndex) {
+        clusterInputIndexMap.put(clusterInputIndex, Pair.of(activityId, activityInputIndex));
+        invertedClusterInputIndexMap.put(Pair.of(activityId, activityInputIndex), clusterInputIndex);
+    }
+
+    /**
+     * get the an internal activity and its input channel of a cluster input channel
+     * 
+     * @param clusterOutputIndex
+     *            the output channel index for the cluster
+     * @return a pair containing the activity id of the corresponding internal activity and the output channel index
+     */
+    public Pair<ActivityId, Integer> getActivityIdInputIndex(int clusterInputIndex) {
+        return clusterInputIndexMap.get(clusterInputIndex);
+    }
+
+    /**
+     * Get the cluster input channel of an input-boundary activity and its input channel
+     * 
+     * @param activityInputChannel
+     *            the input-boundary activity and its input channel
+     * @return the cluster input channel
+     */
+    public int getClusterInputIndex(Pair<ActivityId, Integer> activityInputChannel) {
+        Integer channel = invertedClusterInputIndexMap.get(activityInputChannel);
+        return channel == null ? -1 : channel;
+    }
+
+    /**
+     * Get the cluster output channel of an input-boundary activity and its output channel
+     * 
+     * @param activityOutputChannel
+     *            the output-boundary activity and its output channel
+     * @return the cluster output channel
+     */
+    public int getClusterOutputIndex(Pair<ActivityId, Integer> activityOutputChannel) {
+        Integer channel = invertedClusterOutputIndexMap.get(activityOutputChannel);
+        return channel == null ? -1 : channel;
+    }
+
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivity.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivity.java
new file mode 100644
index 0000000..734ff85
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivity.java
@@ -0,0 +1,177 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.rewriter.runtime;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+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.ActivityCluster;
+import edu.uci.ics.hyracks.api.job.ActivityClusterGraph;
+import edu.uci.ics.hyracks.api.job.ActivityClusterId;
+import edu.uci.ics.hyracks.api.rewriter.OneToOneConnectedActivityCluster;
+
+/**
+ * This class can be used to execute a DAG of activities inside which
+ * there are only one-to-one connectors.
+ * 
+ * @author yingyib
+ */
+public class SuperActivity extends OneToOneConnectedActivityCluster implements IActivity {
+    private static final long serialVersionUID = 1L;
+    private final ActivityId activityId;
+
+    public SuperActivity(ActivityClusterGraph acg, ActivityClusterId id, ActivityId activityId) {
+        super(acg, id);
+        this.activityId = activityId;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+            throws HyracksDataException {
+        final Map<ActivityId, IActivity> startActivities = new HashMap<ActivityId, IActivity>();
+        Map<ActivityId, IActivity> activities = getActivityMap();
+        for (Entry<ActivityId, IActivity> entry : activities.entrySet()) {
+            /**
+             * extract start activities
+             */
+            List<IConnectorDescriptor> conns = getActivityInputMap().get(entry.getKey());
+            if (conns == null || conns.size() == 0) {
+                startActivities.put(entry.getKey(), entry.getValue());
+            }
+        }
+
+        /**
+         * wrap a RecordDescriptorProvider for the super activity
+         */
+        IRecordDescriptorProvider wrappedRecDescProvider = new IRecordDescriptorProvider() {
+
+            @Override
+            public RecordDescriptor getInputRecordDescriptor(ActivityId aid, int inputIndex) {
+                if (startActivities.get(aid) != null) {
+                    /**
+                     * if the activity is a start (input boundary) activity
+                     */
+                    int superActivityInputChannel = SuperActivity.this.getClusterInputIndex(Pair.of(aid, inputIndex));
+                    if (superActivityInputChannel >= 0) {
+                        return recordDescProvider.getInputRecordDescriptor(activityId, superActivityInputChannel);
+                    }
+                }
+                if (SuperActivity.this.getActivityMap().get(aid) != null) {
+                    /**
+                     * if the activity is an internal activity of the super activity
+                     */
+                    IConnectorDescriptor conn = getActivityInputMap().get(aid).get(inputIndex);
+                    return getConnectorRecordDescriptorMap().get(conn.getConnectorId());
+                }
+
+                /**
+                 * the following is for the case where the activity is in other SuperActivities
+                 */
+                ActivityClusterGraph acg = SuperActivity.this.getActivityClusterGraph();
+                for (Entry<ActivityClusterId, ActivityCluster> entry : acg.getActivityClusterMap().entrySet()) {
+                    ActivityCluster ac = entry.getValue();
+                    for (Entry<ActivityId, IActivity> saEntry : ac.getActivityMap().entrySet()) {
+                        SuperActivity sa = (SuperActivity) saEntry.getValue();
+                        if (sa.getActivityMap().get(aid) != null) {
+                            List<IConnectorDescriptor> conns = sa.getActivityInputMap().get(aid);
+                            if (conns != null && conns.size() >= inputIndex) {
+                                IConnectorDescriptor conn = conns.get(inputIndex);
+                                return sa.getConnectorRecordDescriptorMap().get(conn.getConnectorId());
+                            } else {
+                                int superActivityInputChannel = sa.getClusterInputIndex(Pair.of(aid, inputIndex));
+                                if (superActivityInputChannel >= 0) {
+                                    return recordDescProvider.getInputRecordDescriptor(sa.getActivityId(),
+                                            superActivityInputChannel);
+                                }
+                            }
+                        }
+                    }
+                }
+                return null;
+            }
+
+            @Override
+            public RecordDescriptor getOutputRecordDescriptor(ActivityId aid, int outputIndex) {
+                /**
+                 * if the activity is an output-boundary activity
+                 */
+                int superActivityOutputChannel = SuperActivity.this.getClusterOutputIndex(Pair.of(aid, outputIndex));
+                if (superActivityOutputChannel >= 0) {
+                    return recordDescProvider.getOutputRecordDescriptor(activityId, superActivityOutputChannel);
+                }
+
+                if (SuperActivity.this.getActivityMap().get(aid) != null) {
+                    /**
+                     * if the activity is an internal activity of the super activity
+                     */
+                    IConnectorDescriptor conn = getActivityOutputMap().get(aid).get(outputIndex);
+                    return getConnectorRecordDescriptorMap().get(conn.getConnectorId());
+                }
+
+                /**
+                 * the following is for the case where the activity is in other SuperActivities
+                 */
+                ActivityClusterGraph acg = SuperActivity.this.getActivityClusterGraph();
+                for (Entry<ActivityClusterId, ActivityCluster> entry : acg.getActivityClusterMap().entrySet()) {
+                    ActivityCluster ac = entry.getValue();
+                    for (Entry<ActivityId, IActivity> saEntry : ac.getActivityMap().entrySet()) {
+                        SuperActivity sa = (SuperActivity) saEntry.getValue();
+                        if (sa.getActivityMap().get(aid) != null) {
+                            List<IConnectorDescriptor> conns = sa.getActivityOutputMap().get(aid);
+                            if (conns != null && conns.size() >= outputIndex) {
+                                IConnectorDescriptor conn = conns.get(outputIndex);
+                                return sa.getConnectorRecordDescriptorMap().get(conn.getConnectorId());
+                            } else {
+                                superActivityOutputChannel = sa.getClusterOutputIndex(Pair.of(aid, outputIndex));
+                                if (superActivityOutputChannel >= 0) {
+                                    return recordDescProvider.getOutputRecordDescriptor(sa.getActivityId(),
+                                            superActivityOutputChannel);
+                                }
+                            }
+                        }
+                    }
+                }
+                return null;
+            }
+
+        };
+        return new SuperActivityOperatorNodePushable(this, startActivities, ctx, wrappedRecDescProvider, partition,
+                nPartitions);
+    }
+
+    @Override
+    public ActivityId getActivityId() {
+        return activityId;
+    }
+
+    @Override
+    public String toString() {
+        return getActivityMap().values().toString();
+    }
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
new file mode 100644
index 0000000..7d50fa0
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
@@ -0,0 +1,195 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.rewriter.runtime;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Queue;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+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.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+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;
+
+/**
+ * The runtime of a SuperActivity, which internally executes a DAG of one-to-one
+ * connected activities in a single thread.
+ * 
+ * @author yingyib
+ */
+public class SuperActivityOperatorNodePushable implements IOperatorNodePushable {
+    private final Map<ActivityId, IOperatorNodePushable> operatorNodePushables = new HashMap<ActivityId, IOperatorNodePushable>();
+    private final List<IOperatorNodePushable> operatprNodePushablesBFSOrder = new ArrayList<IOperatorNodePushable>();
+    private final Map<ActivityId, IActivity> startActivities;
+    private final SuperActivity parent;
+    private final IHyracksTaskContext ctx;
+    private final IRecordDescriptorProvider recordDescProvider;
+    private final int partition;
+    private final int nPartitions;
+    private int inputArity = 0;
+
+    public SuperActivityOperatorNodePushable(SuperActivity parent, Map<ActivityId, IActivity> startActivities,
+            IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        this.parent = parent;
+        this.startActivities = startActivities;
+        this.ctx = ctx;
+        this.recordDescProvider = recordDescProvider;
+        this.partition = partition;
+        this.nPartitions = nPartitions;
+
+        /**
+         * initialize the writer-relationship for the internal DAG of operator
+         * node pushables
+         */
+        try {
+            init();
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+        /**
+         * initialize operator node pushables in the BFS order
+         */
+        for (IOperatorNodePushable op : operatprNodePushablesBFSOrder) {
+            op.initialize();
+        }
+    }
+
+    public void init() throws HyracksDataException {
+        Map<ActivityId, IOperatorNodePushable> startOperatorNodePushables = new HashMap<ActivityId, IOperatorNodePushable>();
+        Queue<Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>> childQueue = new LinkedList<Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>>();
+        List<IConnectorDescriptor> outputConnectors = null;
+
+        /**
+         * Set up the source operators
+         */
+        for (Entry<ActivityId, IActivity> entry : startActivities.entrySet()) {
+            IOperatorNodePushable opPushable = entry.getValue().createPushRuntime(ctx, recordDescProvider, partition,
+                    nPartitions);
+            startOperatorNodePushables.put(entry.getKey(), opPushable);
+            operatprNodePushablesBFSOrder.add(opPushable);
+            operatorNodePushables.put(entry.getKey(), opPushable);
+            inputArity += opPushable.getInputArity();
+            outputConnectors = parent.getActivityOutputMap().get(entry.getKey());
+            if (outputConnectors != null) {
+                for (IConnectorDescriptor conn : outputConnectors) {
+                    childQueue.add(parent.getConnectorActivityMap().get(conn.getConnectorId()));
+                }
+            }
+        }
+
+        /**
+         * Using BFS (breadth-first search) to construct to runtime execution
+         * DAG;
+         */
+        while (childQueue.size() > 0) {
+            /**
+             * expend the executing activities further to the downstream
+             */
+            if (outputConnectors != null && outputConnectors.size() > 0) {
+                for (IConnectorDescriptor conn : outputConnectors) {
+                    if (conn != null) {
+                        childQueue.add(parent.getConnectorActivityMap().get(conn.getConnectorId()));
+                    }
+                }
+            }
+
+            /**
+             * construct the source to destination information
+             */
+            Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>> channel = childQueue.poll();
+            ActivityId sourceId = channel.getLeft().getLeft().getActivityId();
+            int outputChannel = channel.getLeft().getRight();
+            ActivityId destId = channel.getRight().getLeft().getActivityId();
+            int inputChannel = channel.getRight().getRight();
+            IOperatorNodePushable sourceOp = operatorNodePushables.get(sourceId);
+            IOperatorNodePushable destOp = operatorNodePushables.get(destId);
+            if (destOp == null) {
+                destOp = channel.getRight().getLeft()
+                        .createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+                operatprNodePushablesBFSOrder.add(destOp);
+                operatorNodePushables.put(destId, destOp);
+            }
+
+            /**
+             * construct the dataflow connection from a producer to a consumer
+             */
+            sourceOp.setOutputFrameWriter(outputChannel, destOp.getInputFrameWriter(inputChannel),
+                    recordDescProvider.getInputRecordDescriptor(destId, inputChannel));
+
+            /**
+             * traverse to the child of the current activity
+             */
+            outputConnectors = parent.getActivityOutputMap().get(destId);
+        }
+    }
+
+    @Override
+    public void deinitialize() throws HyracksDataException {
+        /**
+         * de-initialize operator node pushables
+         */
+        for (IOperatorNodePushable op : operatprNodePushablesBFSOrder) {
+            op.deinitialize();
+        }
+    }
+
+    @Override
+    public int getInputArity() {
+        return inputArity;
+    }
+
+    @Override
+    public void setOutputFrameWriter(int clusterOutputIndex, IFrameWriter writer, RecordDescriptor recordDesc) {
+        /**
+         * set the right output frame writer
+         */
+        Pair<ActivityId, Integer> activityIdOutputIndex = parent.getActivityIdOutputIndex(clusterOutputIndex);
+        IOperatorNodePushable opPushable = operatorNodePushables.get(activityIdOutputIndex.getLeft());
+        opPushable.setOutputFrameWriter(activityIdOutputIndex.getRight(), writer, recordDesc);
+    }
+
+    @Override
+    public IFrameWriter getInputFrameWriter(final int index) {
+        /**
+         * get the right IFrameWriter from the cluster input index
+         */
+        Pair<ActivityId, Integer> activityIdInputIndex = parent.getActivityIdInputIndex(index);
+        IOperatorNodePushable operatorNodePushable = operatorNodePushables.get(activityIdInputIndex.getLeft());
+        IFrameWriter writer = operatorNodePushable.getInputFrameWriter(activityIdInputIndex.getRight());
+        return writer;
+    }
+
+    @Override
+    public String getDisplayName() {
+        return "Super Activity " + parent.getActivityMap().values().toString();
+    }
+
+}
diff --git a/hyracks/hyracks-cli/pom.xml b/hyracks/hyracks-cli/pom.xml
index 3933be0..991ce2a 100644
--- a/hyracks/hyracks-cli/pom.xml
+++ b/hyracks/hyracks-cli/pom.xml
@@ -18,8 +18,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
       <plugin>
@@ -41,6 +42,7 @@
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
             <configuration>
diff --git a/hyracks/hyracks-client/pom.xml b/hyracks/hyracks-client/pom.xml
new file mode 100644
index 0000000..854a009
--- /dev/null
+++ b/hyracks/hyracks-client/pom.xml
@@ -0,0 +1,46 @@
+<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>
+  <artifactId>hyracks-client</artifactId>
+  <name>hyracks-client</name>
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.2.3-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.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+ <dependency>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-api</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-net</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-comm</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+  <dependency>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-dataflow-common</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+  </dependencies>
+</project>
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
new file mode 100644
index 0000000..8be4a8c
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.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.client.dataset;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+
+public class DatasetClientContext implements IHyracksCommonContext {
+    private final int frameSize;
+
+    public DatasetClientContext(int frameSize) {
+        this.frameSize = frameSize;
+    }
+
+    @Override
+    public int getFrameSize() {
+        return frameSize;
+    }
+
+    @Override
+    public IIOManager getIOManager() {
+        return null;
+    }
+
+    @Override
+    public ByteBuffer allocateFrame() {
+        return ByteBuffer.allocate(frameSize);
+    }
+
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDataset.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDataset.java
new file mode 100644
index 0000000..6866e46
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDataset.java
@@ -0,0 +1,56 @@
+/*
+ * 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.client.dataset;
+
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.client.net.ClientNetworkManager;
+
+public class HyracksDataset implements IHyracksDataset {
+    private final IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection;
+
+    private final ClientNetworkManager netManager;
+
+    private final DatasetClientContext datasetClientCtx;
+
+    public HyracksDataset(IHyracksClientConnection hcc, int frameSize, int nReaders) throws Exception {
+        NetworkAddress ddsAddress = hcc.getDatasetDirectoryServiceInfo();
+        datasetDirectoryServiceConnection = new HyracksDatasetDirectoryServiceConnection(new String(
+                ddsAddress.getIpAddress()), ddsAddress.getPort());
+
+        netManager = new ClientNetworkManager(nReaders);
+        netManager.start();
+
+        datasetClientCtx = new DatasetClientContext(frameSize);
+    }
+
+    @Override
+    public IHyracksDatasetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
+        IHyracksDatasetReader reader = null;
+        try {
+            reader = new HyracksDatasetReader(datasetDirectoryServiceConnection, netManager, datasetClientCtx, jobId,
+                    resultSetId);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+        return reader;
+    }
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java
new file mode 100644
index 0000000..095fd7d
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.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.client.dataset;
+
+import java.net.InetSocketAddress;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceInterface;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
+import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
+import edu.uci.ics.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
+
+//TODO(madhusudancs): Should this implementation be moved to edu.uci.ics.hyracks.client?
+public class HyracksDatasetDirectoryServiceConnection implements IHyracksDatasetDirectoryServiceConnection {
+    private final IPCSystem ipc;
+    private final IHyracksDatasetDirectoryServiceInterface ddsi;
+
+    public HyracksDatasetDirectoryServiceConnection(String ddsHost, int ddsPort) throws Exception {
+        RPCInterface rpci = new RPCInterface();
+        ipc = new IPCSystem(new InetSocketAddress(0), rpci, new JavaSerializationBasedPayloadSerializerDeserializer());
+        ipc.start();
+        IIPCHandle ddsIpchandle = ipc.getHandle(new InetSocketAddress(ddsHost, ddsPort));
+        this.ddsi = new HyracksDatasetDirectoryServiceInterfaceRemoteProxy(ddsIpchandle, rpci);
+    }
+
+    @Override
+    public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception {
+        return ddsi.getDatasetResultStatus(jobId, rsId);
+    }
+
+    @Override
+    public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownRecords) throws Exception {
+        return ddsi.getDatasetResultLocations(jobId, rsId, knownRecords);
+    }
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java
new file mode 100644
index 0000000..47cdf97
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.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.client.dataset;
+
+import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceInterface;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
+
+//TODO(madhusudancs): Should this implementation be moved to edu.uci.ics.hyracks.client?
+public class HyracksDatasetDirectoryServiceInterfaceRemoteProxy implements IHyracksDatasetDirectoryServiceInterface {
+    private final IIPCHandle ipcHandle;
+
+    private final RPCInterface rpci;
+
+    public HyracksDatasetDirectoryServiceInterfaceRemoteProxy(IIPCHandle ipcHandle, RPCInterface rpci) {
+        this.ipcHandle = ipcHandle;
+        this.rpci = rpci;
+    }
+
+    @Override
+    public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception {
+        HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction gdrlf = new HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction(
+                jobId, rsId);
+        return (Status) rpci.call(ipcHandle, gdrlf);
+    }
+
+    @Override
+    public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownRecords) throws Exception {
+        HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction gdrlf = new HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction(
+                jobId, rsId, knownRecords);
+        return (DatasetDirectoryRecord[]) rpci.call(ipcHandle, gdrlf);
+    }
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
new file mode 100644
index 0000000..78bcf20
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
@@ -0,0 +1,255 @@
+/*
+ * 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.client.dataset;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IDatasetInputChannelMonitor;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.client.net.ClientNetworkManager;
+import edu.uci.ics.hyracks.comm.channels.DatasetNetworkInputChannel;
+
+// TODO(madhusudancs): Should this implementation be moved to edu.uci.ics.hyracks.client?
+public class HyracksDatasetReader implements IHyracksDatasetReader {
+    private static final Logger LOGGER = Logger.getLogger(HyracksDatasetReader.class.getName());
+
+    private final IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection;
+
+    private final ClientNetworkManager netManager;
+
+    private final DatasetClientContext datasetClientCtx;
+
+    private JobId jobId;
+
+    private ResultSetId resultSetId;
+
+    private DatasetDirectoryRecord[] knownRecords;
+
+    private IDatasetInputChannelMonitor[] monitors;
+
+    private int lastReadPartition;
+
+    private IDatasetInputChannelMonitor lastMonitor;
+
+    private DatasetNetworkInputChannel resultChannel;
+
+    private static int NUM_READ_BUFFERS = 1;
+
+    public HyracksDatasetReader(IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection,
+            ClientNetworkManager netManager, DatasetClientContext datasetClientCtx, JobId jobId, ResultSetId resultSetId)
+            throws Exception {
+        this.datasetDirectoryServiceConnection = datasetDirectoryServiceConnection;
+        this.netManager = netManager;
+        this.datasetClientCtx = datasetClientCtx;
+        this.jobId = jobId;
+        this.resultSetId = resultSetId;
+        knownRecords = null;
+        monitors = null;
+        lastReadPartition = -1;
+        lastMonitor = null;
+        resultChannel = null;
+    }
+
+    @Override
+    public Status getResultStatus() {
+        Status status = null;
+        try {
+            status = datasetDirectoryServiceConnection.getDatasetResultStatus(jobId, resultSetId);
+        } catch (Exception e) {
+            // TODO(madhusudancs): Decide what to do in case of error
+        }
+        return status;
+    }
+
+    @Override
+    public int read(ByteBuffer buffer) throws HyracksDataException {
+        ByteBuffer readBuffer;
+        int readSize = 0;
+
+        if (lastReadPartition == -1) {
+            while (knownRecords == null || knownRecords[0] == null) {
+                try {
+                    knownRecords = datasetDirectoryServiceConnection.getDatasetResultLocations(jobId, resultSetId,
+                            knownRecords);
+                    lastReadPartition = 0;
+                    resultChannel = new DatasetNetworkInputChannel(netManager,
+                            getSocketAddress(knownRecords[lastReadPartition]), jobId, lastReadPartition,
+                            NUM_READ_BUFFERS);
+                    lastMonitor = getMonitor(lastReadPartition);
+                    resultChannel.open(datasetClientCtx);
+                    resultChannel.registerMonitor(lastMonitor);
+                } catch (HyracksException e) {
+                    throw new HyracksDataException(e);
+                } catch (UnknownHostException e) {
+                    throw new HyracksDataException(e);
+                } catch (Exception e) {
+                    // Do nothing here.
+                }
+            }
+        }
+
+        while (readSize <= 0 && !((lastReadPartition == knownRecords.length - 1) && (lastMonitor.eosReached()))) {
+            synchronized (lastMonitor) {
+                while (lastMonitor.getNFramesAvailable() <= 0 && !lastMonitor.eosReached()) {
+                    try {
+                        lastMonitor.wait();
+                    } catch (InterruptedException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+            }
+
+            if (lastMonitor.getNFramesAvailable() <= 0 && lastMonitor.eosReached()) {
+                knownRecords[lastReadPartition].readEOS();
+                if ((lastReadPartition == knownRecords.length - 1)) {
+                    break;
+                } else {
+                    try {
+                        lastReadPartition++;
+                        while (knownRecords[lastReadPartition] == null) {
+                            try {
+                                knownRecords = datasetDirectoryServiceConnection.getDatasetResultLocations(jobId,
+                                        resultSetId, knownRecords);
+                            } catch (Exception e) {
+                                // Do nothing here.
+                            }
+                        }
+
+                        resultChannel = new DatasetNetworkInputChannel(netManager,
+                                getSocketAddress(knownRecords[lastReadPartition]), jobId, lastReadPartition,
+                                NUM_READ_BUFFERS);
+                        lastMonitor = getMonitor(lastReadPartition);
+                        resultChannel.open(datasetClientCtx);
+                        resultChannel.registerMonitor(lastMonitor);
+                    } catch (HyracksException e) {
+                        throw new HyracksDataException(e);
+                    } catch (UnknownHostException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+            } else {
+                readBuffer = resultChannel.getNextBuffer();
+                lastMonitor.notifyFrameRead();
+                if (readBuffer != null) {
+                    buffer.put(readBuffer);
+                    buffer.flip();
+                    readSize = buffer.limit();
+                    resultChannel.recycleBuffer(readBuffer);
+                }
+            }
+        }
+
+        return readSize;
+    }
+
+    private boolean nullExists(DatasetDirectoryRecord[] locations) {
+        if (locations == null) {
+            return true;
+        }
+        for (int i = 0; i < locations.length; i++) {
+            if (locations[i] == null) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private SocketAddress getSocketAddress(DatasetDirectoryRecord addr) throws UnknownHostException {
+        NetworkAddress netAddr = addr.getNetworkAddress();
+        return new InetSocketAddress(InetAddress.getByAddress(netAddr.getIpAddress()), netAddr.getPort());
+    }
+
+    private IDatasetInputChannelMonitor getMonitor(int partition) throws HyracksException {
+        if (knownRecords == null || knownRecords[partition] == null) {
+            throw new HyracksException("Accessing monitors before the obtaining the corresponding addresses.");
+        }
+        if (monitors == null) {
+            monitors = new DatasetInputChannelMonitor[knownRecords.length];
+        }
+        if (monitors[partition] == null) {
+            monitors[partition] = new DatasetInputChannelMonitor();
+        }
+        return monitors[partition];
+    }
+
+    private class DatasetInputChannelMonitor implements IDatasetInputChannelMonitor {
+        private final AtomicInteger nAvailableFrames;
+
+        private final AtomicBoolean eos;
+
+        private final AtomicBoolean failed;
+
+        public DatasetInputChannelMonitor() {
+            nAvailableFrames = new AtomicInteger(0);
+            eos = new AtomicBoolean(false);
+            failed = new AtomicBoolean(false);
+        }
+
+        @Override
+        public synchronized void notifyFailure(IInputChannel channel) {
+            failed.set(true);
+            notifyAll();
+        }
+
+        @Override
+        public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
+            nAvailableFrames.addAndGet(nFrames);
+            notifyAll();
+        }
+
+        @Override
+        public synchronized void notifyEndOfStream(IInputChannel channel) {
+            eos.set(true);
+            notifyAll();
+        }
+
+        @Override
+        public synchronized boolean eosReached() {
+            return eos.get();
+        }
+
+        @Override
+        public synchronized boolean failed() {
+            return failed.get();
+        }
+
+        @Override
+        public synchronized int getNFramesAvailable() {
+            return nAvailableFrames.get();
+        }
+
+        @Override
+        public synchronized void notifyFrameRead() {
+            nAvailableFrames.decrementAndGet();
+        }
+
+    }
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/net/ClientNetworkManager.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/net/ClientNetworkManager.java
new file mode 100644
index 0000000..7aef8b9
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/net/ClientNetworkManager.java
@@ -0,0 +1,56 @@
+/*
+ * 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.client.net;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+
+import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
+
+public class ClientNetworkManager implements IChannelConnectionFactory {
+    private static final int MAX_CONNECTION_ATTEMPTS = 5;
+
+    private final MuxDemux md;
+
+    public ClientNetworkManager(int nThreads) throws IOException {
+        /* This is a connect only socket and does not listen to any incoming connections, so pass null to
+         * localAddress and listener.
+         */
+        md = new MuxDemux(null, null, nThreads, MAX_CONNECTION_ATTEMPTS);
+    }
+
+    public void start() throws IOException {
+        md.start();
+    }
+
+    public void stop() {
+
+    }
+
+    public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException {
+        MultiplexedConnection mConn = md.connect((InetSocketAddress) remoteAddress);
+        return mConn.openChannel();
+    }
+
+    public MuxDemuxPerformanceCounters getPerformanceCounters() {
+        return md.getPerformanceCounters();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-comm/pom.xml b/hyracks/hyracks-comm/pom.xml
new file mode 100644
index 0000000..c3583699
--- /dev/null
+++ b/hyracks/hyracks-comm/pom.xml
@@ -0,0 +1,36 @@
+<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>
+  <artifactId>hyracks-comm</artifactId>
+  <name>hyracks-comm</name>
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.2.3-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.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+ <dependency>
+ 	<groupId>edu.uci.ics.hyracks</groupId>
+ 	<artifactId>hyracks-api</artifactId>
+ 	<version>0.2.3-SNAPSHOT</version>
+ </dependency>
+  <dependency>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-net</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+  </dependencies>
+</project>
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
similarity index 72%
copy from hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
copy to hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
index 1d5af84..fac2949 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.nc.net;
+package edu.uci.ics.hyracks.comm.channels;
 
 import java.net.SocketAddress;
 import java.nio.ByteBuffer;
@@ -23,21 +23,25 @@
 
 import edu.uci.ics.hyracks.api.channels.IInputChannel;
 import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
 import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
 import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
 
-public class NetworkInputChannel implements IInputChannel {
-    private static final Logger LOGGER = Logger.getLogger(NetworkInputChannel.class.getName());
+public class DatasetNetworkInputChannel implements IInputChannel {
+    private static final Logger LOGGER = Logger.getLogger(DatasetNetworkInputChannel.class.getName());
 
-    private final NetworkManager netManager;
+    static final int INITIAL_MESSAGE_SIZE = 20;
+
+    private final IChannelConnectionFactory netManager;
 
     private final SocketAddress remoteAddress;
 
-    private final PartitionId partitionId;
+    private final JobId jobId;
+
+    private final int partition;
 
     private final Queue<ByteBuffer> fullQueue;
 
@@ -49,11 +53,12 @@
 
     private Object attachment;
 
-    public NetworkInputChannel(NetworkManager netManager, SocketAddress remoteAddress, PartitionId partitionId,
-            int nBuffers) {
+    public DatasetNetworkInputChannel(IChannelConnectionFactory netManager, SocketAddress remoteAddress, JobId jobId,
+            int partition, int nBuffers) {
         this.netManager = netManager;
         this.remoteAddress = remoteAddress;
-        this.partitionId = partitionId;
+        this.jobId = jobId;
+        this.partition = partition;
         fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
         this.nBuffers = nBuffers;
     }
@@ -85,7 +90,7 @@
     }
 
     @Override
-    public void open(IHyracksTaskContext ctx) throws HyracksDataException {
+    public void open(IHyracksCommonContext ctx) throws HyracksDataException {
         try {
             ccb = netManager.connect(remoteAddress);
         } catch (Exception e) {
@@ -96,14 +101,13 @@
         for (int i = 0; i < nBuffers; ++i) {
             ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
         }
-        ByteBuffer writeBuffer = ByteBuffer.allocate(NetworkManager.INITIAL_MESSAGE_SIZE);
-        writeBuffer.putLong(partitionId.getJobId().getId());
-        writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
-        writeBuffer.putInt(partitionId.getSenderIndex());
-        writeBuffer.putInt(partitionId.getReceiverIndex());
+        ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
+        writeBuffer.putLong(jobId.getId());
+        writeBuffer.putInt(partition);
         writeBuffer.flip();
         if (LOGGER.isLoggable(Level.FINE)) {
-            LOGGER.fine("Sending partition request: " + partitionId + " on channel: " + ccb);
+            LOGGER.fine("Sending partition request for JobId: " + jobId + " partition: " + partition + " on channel: "
+                    + ccb);
         }
         ccb.getWriteInterface().getFullBufferAcceptor().accept(writeBuffer);
         ccb.getWriteInterface().getFullBufferAcceptor().close();
@@ -118,17 +122,17 @@
         @Override
         public void accept(ByteBuffer buffer) {
             fullQueue.add(buffer);
-            monitor.notifyDataAvailability(NetworkInputChannel.this, 1);
+            monitor.notifyDataAvailability(DatasetNetworkInputChannel.this, 1);
         }
 
         @Override
         public void close() {
-            monitor.notifyEndOfStream(NetworkInputChannel.this);
+            monitor.notifyEndOfStream(DatasetNetworkInputChannel.this);
         }
 
         @Override
         public void error(int ecode) {
-            monitor.notifyFailure(NetworkInputChannel.this);
+            monitor.notifyFailure(DatasetNetworkInputChannel.this);
         }
     }
 
diff --git a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/IChannelConnectionFactory.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/IChannelConnectionFactory.java
new file mode 100644
index 0000000..33179ba
--- /dev/null
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/IChannelConnectionFactory.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.comm.channels;
+
+import java.net.SocketAddress;
+
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+
+public interface IChannelConnectionFactory {
+    public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException;
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
similarity index 89%
rename from hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
rename to hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
index 1d5af84..aa37b16 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.nc.net;
+package edu.uci.ics.hyracks.comm.channels;
 
 import java.net.SocketAddress;
 import java.nio.ByteBuffer;
@@ -23,7 +23,7 @@
 
 import edu.uci.ics.hyracks.api.channels.IInputChannel;
 import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
 import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
@@ -33,7 +33,9 @@
 public class NetworkInputChannel implements IInputChannel {
     private static final Logger LOGGER = Logger.getLogger(NetworkInputChannel.class.getName());
 
-    private final NetworkManager netManager;
+    static final int INITIAL_MESSAGE_SIZE = 20;
+
+    private final IChannelConnectionFactory netManager;
 
     private final SocketAddress remoteAddress;
 
@@ -49,8 +51,8 @@
 
     private Object attachment;
 
-    public NetworkInputChannel(NetworkManager netManager, SocketAddress remoteAddress, PartitionId partitionId,
-            int nBuffers) {
+    public NetworkInputChannel(IChannelConnectionFactory netManager, SocketAddress remoteAddress,
+            PartitionId partitionId, int nBuffers) {
         this.netManager = netManager;
         this.remoteAddress = remoteAddress;
         this.partitionId = partitionId;
@@ -85,7 +87,7 @@
     }
 
     @Override
-    public void open(IHyracksTaskContext ctx) throws HyracksDataException {
+    public void open(IHyracksCommonContext ctx) throws HyracksDataException {
         try {
             ccb = netManager.connect(remoteAddress);
         } catch (Exception e) {
@@ -96,7 +98,7 @@
         for (int i = 0; i < nBuffers; ++i) {
             ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
         }
-        ByteBuffer writeBuffer = ByteBuffer.allocate(NetworkManager.INITIAL_MESSAGE_SIZE);
+        ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
         writeBuffer.putLong(partitionId.getJobId().getId());
         writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
         writeBuffer.putInt(partitionId.getSenderIndex());
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
similarity index 92%
rename from hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
rename to hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
index 9024e18..812a2de 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
@@ -12,14 +12,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.nc.net;
+package edu.uci.ics.hyracks.comm.channels;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayDeque;
 import java.util.Deque;
 
 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.net.buffers.IBufferAcceptor;
 import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
@@ -40,9 +39,9 @@
         ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
     }
 
-    public void setTaskContext(IHyracksTaskContext ctx) {
+    public void setFrameSize(int frameSize) {
         for (int i = 0; i < nBuffers; ++i) {
-            emptyStack.push(ByteBuffer.allocateDirect(ctx.getFrameSize()));
+            emptyStack.push(ByteBuffer.allocateDirect(frameSize));
         }
     }
 
@@ -87,7 +86,7 @@
         ccb.getWriteInterface().getFullBufferAcceptor().close();
     }
 
-    void abort() {
+    public void abort() {
         ccb.getWriteInterface().getFullBufferAcceptor().error(1);
         synchronized (NetworkOutputChannel.this) {
             aborted = true;
diff --git a/hyracks/hyracks-control/hyracks-control-cc/pom.xml b/hyracks/hyracks-control/hyracks-control-cc/pom.xml
index c7eedb3..d644673 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/pom.xml
+++ b/hyracks/hyracks-control/hyracks-control-cc/pom.xml
@@ -15,8 +15,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index 5a33891..82457fe 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -35,12 +35,17 @@
 import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
 import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
 import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.context.ICCContext;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.api.topology.ClusterTopology;
 import edu.uci.ics.hyracks.api.topology.TopologyDefinitionParser;
 import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.cc.dataset.DatasetDirectoryService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
 import edu.uci.ics.hyracks.control.cc.web.WebServer;
 import edu.uci.ics.hyracks.control.cc.work.ApplicationCreateWork;
@@ -48,17 +53,23 @@
 import edu.uci.ics.hyracks.control.cc.work.ApplicationMessageWork;
 import edu.uci.ics.hyracks.control.cc.work.ApplicationStartWork;
 import edu.uci.ics.hyracks.control.cc.work.ApplicationStateChangeWork;
+import edu.uci.ics.hyracks.control.cc.work.GetDatasetDirectoryServiceInfoWork;
 import edu.uci.ics.hyracks.control.cc.work.GetIpAddressNodeNameMapWork;
 import edu.uci.ics.hyracks.control.cc.work.GetJobStatusWork;
 import edu.uci.ics.hyracks.control.cc.work.GetNodeControllersInfoWork;
+import edu.uci.ics.hyracks.control.cc.work.GetResultPartitionLocationsWork;
+import edu.uci.ics.hyracks.control.cc.work.GetResultStatusWork;
 import edu.uci.ics.hyracks.control.cc.work.JobStartWork;
 import edu.uci.ics.hyracks.control.cc.work.JobletCleanupNotificationWork;
 import edu.uci.ics.hyracks.control.cc.work.NodeHeartbeatWork;
 import edu.uci.ics.hyracks.control.cc.work.RegisterNodeWork;
 import edu.uci.ics.hyracks.control.cc.work.RegisterPartitionAvailibilityWork;
 import edu.uci.ics.hyracks.control.cc.work.RegisterPartitionRequestWork;
+import edu.uci.ics.hyracks.control.cc.work.RegisterResultPartitionLocationWork;
 import edu.uci.ics.hyracks.control.cc.work.RemoveDeadNodesWork;
 import edu.uci.ics.hyracks.control.cc.work.ReportProfilesWork;
+import edu.uci.ics.hyracks.control.cc.work.ReportResultPartitionFailureWork;
+import edu.uci.ics.hyracks.control.cc.work.ReportResultPartitionWriteCompletionWork;
 import edu.uci.ics.hyracks.control.cc.work.TaskCompleteWork;
 import edu.uci.ics.hyracks.control.cc.work.TaskFailureWork;
 import edu.uci.ics.hyracks.control.cc.work.UnregisterNodeWork;
@@ -115,6 +126,8 @@
 
     private final DeadNodeSweeper sweeper;
 
+    private final IDatasetDirectoryService datasetDirectoryService;
+
     private long jobCounter;
 
     public ClusterControllerService(final CCConfig ccConfig) throws Exception {
@@ -162,6 +175,7 @@
             }
         };
         sweeper = new DeadNodeSweeper();
+        datasetDirectoryService = new DatasetDirectoryService();
         jobCounter = 0;
     }
 
@@ -264,6 +278,10 @@
         return clusterIPC;
     }
 
+    public NetworkAddress getDatasetDirectoryServiceInfo() {
+        return new NetworkAddress(ccConfig.clientNetIpAddress.getBytes(), ccConfig.clientNetPort);
+    }
+
     private class DeadNodeSweeper extends TimerTask {
         @Override
         public void run() {
@@ -271,6 +289,10 @@
         }
     }
 
+    public IDatasetDirectoryService getDatasetDirectoryService() {
+        return datasetDirectoryService;
+    }
+
     private class HyracksClientInterfaceIPCI implements IIPCI {
         @Override
         public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception) {
@@ -321,6 +343,27 @@
                     return;
                 }
 
+                case GET_DATASET_DIRECTORY_SERIVICE_INFO: {
+                    workQueue.schedule(new GetDatasetDirectoryServiceInfoWork(ClusterControllerService.this,
+                            new IPCResponder<NetworkAddress>(handle, mid)));
+                    return;
+                }
+
+                case GET_DATASET_RESULT_STATUS: {
+                    HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction gdrlf = (HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction) fn;
+                    workQueue.schedule(new GetResultStatusWork(ClusterControllerService.this, gdrlf.getJobId(), gdrlf
+                            .getResultSetId(), new IPCResponder<Status>(handle, mid)));
+                    return;
+                }
+
+                case GET_DATASET_RESULT_LOCATIONS: {
+                    HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction gdrlf = (HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction) fn;
+                    workQueue.schedule(new GetResultPartitionLocationsWork(ClusterControllerService.this, gdrlf
+                            .getJobId(), gdrlf.getResultSetId(), gdrlf.getKnownRecords(),
+                            new IPCResponder<DatasetDirectoryRecord[]>(handle, mid)));
+                    return;
+                }
+
                 case WAIT_FOR_COMPLETION: {
                     HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = (HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
                     workQueue.schedule(new WaitForJobCompletionWork(ClusterControllerService.this, wfcf.getJobId(),
@@ -416,6 +459,28 @@
                     return;
                 }
 
+                case REGISTER_RESULT_PARTITION_LOCATION: {
+                    CCNCFunctions.RegisterResultPartitionLocationFunction rrplf = (CCNCFunctions.RegisterResultPartitionLocationFunction) fn;
+                    workQueue.schedule(new RegisterResultPartitionLocationWork(ClusterControllerService.this, rrplf
+                            .getJobId(), rrplf.getResultSetId(), rrplf.getOrderedResult(), rrplf.getPartition(), rrplf
+                            .getNPartitions(), rrplf.getNetworkAddress()));
+                    return;
+                }
+
+                case REPORT_RESULT_PARTITION_WRITE_COMPLETION: {
+                    CCNCFunctions.ReportResultPartitionWriteCompletionFunction rrplf = (CCNCFunctions.ReportResultPartitionWriteCompletionFunction) fn;
+                    workQueue.schedule(new ReportResultPartitionWriteCompletionWork(ClusterControllerService.this,
+                            rrplf.getJobId(), rrplf.getResultSetId(), rrplf.getPartition()));
+                    return;
+                }
+
+                case REPORT_RESULT_PARTITION_FAILURE: {
+                    CCNCFunctions.ReportResultPartitionFailureFunction rrplf = (CCNCFunctions.ReportResultPartitionFailureFunction) fn;
+                    workQueue.schedule(new ReportResultPartitionFailureWork(ClusterControllerService.this, rrplf
+                            .getJobId(), rrplf.getResultSetId(), rrplf.getPartition()));
+                    return;
+                }
+
                 case APPLICATION_STATE_CHANGE_RESPONSE: {
                     CCNCFunctions.ApplicationStateChangeResponseFunction astrf = (CCNCFunctions.ApplicationStateChangeResponseFunction) fn;
                     workQueue.schedule(new ApplicationStateChangeWork(ClusterControllerService.this, astrf));
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
index c17acd0..c96a319 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
@@ -41,6 +41,8 @@
 
     private final NetworkAddress dataPort;
 
+    private final NetworkAddress datasetPort;
+
     private final Set<JobId> activeJobIds;
 
     private final String osName;
@@ -107,6 +109,14 @@
 
     private final long[] netSignalingBytesWritten;
 
+    private final long[] datasetNetPayloadBytesRead;
+
+    private final long[] datasetNetPayloadBytesWritten;
+
+    private final long[] datasetNetSignalingBytesRead;
+
+    private final long[] datasetNetSignalingBytesWritten;
+
     private final long[] ipcMessagesSent;
 
     private final long[] ipcMessageBytesSent;
@@ -123,6 +133,7 @@
         this.nodeController = nodeController;
         ncConfig = reg.getNCConfig();
         dataPort = reg.getDataPort();
+        datasetPort = reg.getDatasetPort();
         activeJobIds = new HashSet<JobId>();
 
         osName = reg.getOSName();
@@ -164,6 +175,10 @@
         netPayloadBytesWritten = new long[RRD_SIZE];
         netSignalingBytesRead = new long[RRD_SIZE];
         netSignalingBytesWritten = new long[RRD_SIZE];
+        datasetNetPayloadBytesRead = new long[RRD_SIZE];
+        datasetNetPayloadBytesWritten = new long[RRD_SIZE];
+        datasetNetSignalingBytesRead = new long[RRD_SIZE];
+        datasetNetSignalingBytesWritten = new long[RRD_SIZE];
         ipcMessagesSent = new long[RRD_SIZE];
         ipcMessageBytesSent = new long[RRD_SIZE];
         ipcMessagesReceived = new long[RRD_SIZE];
@@ -196,6 +211,10 @@
         netPayloadBytesWritten[rrdPtr] = hbData.netPayloadBytesWritten;
         netSignalingBytesRead[rrdPtr] = hbData.netSignalingBytesRead;
         netSignalingBytesWritten[rrdPtr] = hbData.netSignalingBytesWritten;
+        datasetNetPayloadBytesRead[rrdPtr] = hbData.datasetNetPayloadBytesRead;
+        datasetNetPayloadBytesWritten[rrdPtr] = hbData.datasetNetPayloadBytesWritten;
+        datasetNetSignalingBytesRead[rrdPtr] = hbData.datasetNetSignalingBytesRead;
+        datasetNetSignalingBytesWritten[rrdPtr] = hbData.datasetNetSignalingBytesWritten;
         ipcMessagesSent[rrdPtr] = hbData.ipcMessagesSent;
         ipcMessageBytesSent[rrdPtr] = hbData.ipcMessageBytesSent;
         ipcMessagesReceived[rrdPtr] = hbData.ipcMessagesReceived;
@@ -227,6 +246,10 @@
         return dataPort;
     }
 
+    public NetworkAddress getDatasetPort() {
+        return datasetPort;
+    }
+
     public JSONObject toSummaryJSON() throws JSONException {
         JSONObject o = new JSONObject();
         o.put("node-id", ncConfig.nodeId);
@@ -271,6 +294,10 @@
         o.put("net-payload-bytes-written", netPayloadBytesWritten);
         o.put("net-signaling-bytes-read", netSignalingBytesRead);
         o.put("net-signaling-bytes-written", netSignalingBytesWritten);
+        o.put("dataset-net-payload-bytes-read", datasetNetPayloadBytesRead);
+        o.put("dataset-net-payload-bytes-written", datasetNetPayloadBytesWritten);
+        o.put("dataset-net-signaling-bytes-read", datasetNetSignalingBytesRead);
+        o.put("dataset-net-signaling-bytes-written", datasetNetSignalingBytesWritten);
         o.put("ipc-messages-sent", ipcMessagesSent);
         o.put("ipc-message-bytes-sent", ipcMessageBytesSent);
         o.put("ipc-messages-received", ipcMessagesReceived);
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java
new file mode 100644
index 0000000..13d0c30
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java
@@ -0,0 +1,241 @@
+/*
+ * 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.control.cc.dataset;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+/**
+ * TODO(madhusudancs): The potential perils of this global dataset directory service implementation is that, the jobs
+ * location information is never evicted from the memory and the memory usage grows as the number of jobs in the system
+ * grows. What we should possibly do is, add an API call for the client to say that it received everything it has to for
+ * the job (after it receives all the results) completely. Then we can just get rid of the location information for that
+ * job.
+ */
+public class DatasetDirectoryService implements IDatasetDirectoryService {
+    private final Map<JobId, Map<ResultSetId, ResultSetMetaData>> jobResultLocationsMap;
+
+    public DatasetDirectoryService() {
+        jobResultLocationsMap = new HashMap<JobId, Map<ResultSetId, ResultSetMetaData>>();
+    }
+
+    @Override
+    public synchronized void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult,
+            int partition, int nPartitions, NetworkAddress networkAddress) {
+        Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
+        if (rsMap == null) {
+            rsMap = new HashMap<ResultSetId, ResultSetMetaData>();
+            jobResultLocationsMap.put(jobId, rsMap);
+        }
+
+        ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+        if (resultSetMetaData == null) {
+            resultSetMetaData = new ResultSetMetaData(orderedResult, new DatasetDirectoryRecord[nPartitions]);
+            rsMap.put(rsId, resultSetMetaData);
+        }
+
+        DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+        if (records[partition] == null) {
+            records[partition] = new DatasetDirectoryRecord();
+        }
+        records[partition].setNetworkAddress(networkAddress);
+        records[partition].start();
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) {
+        DatasetDirectoryRecord ddr = getDatasetDirectoryRecord(jobId, rsId, partition);
+        ddr.writeEOS();
+    }
+
+    @Override
+    public synchronized void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) {
+        DatasetDirectoryRecord ddr = getDatasetDirectoryRecord(jobId, rsId, partition);
+        ddr.fail();
+    }
+
+    @Override
+    public synchronized Status getResultStatus(JobId jobId, ResultSetId rsId) throws HyracksDataException {
+        Map<ResultSetId, ResultSetMetaData> rsMap;
+        while ((rsMap = jobResultLocationsMap.get(jobId)) == null) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+        if (resultSetMetaData == null || resultSetMetaData.getRecords() == null) {
+            throw new HyracksDataException("ResultSet locations uninitialized when it is expected to be initialized.");
+        }
+        DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+
+        ArrayList<Status> statuses = new ArrayList<Status>(records.length);
+        for (int i = 0; i < records.length; i++) {
+            statuses.add(records[i].getStatus());
+        }
+
+        // Default status is idle
+        Status status = Status.IDLE;
+        if (statuses.contains(Status.FAILED)) {
+            // Even if there is at least one failed entry we should return failed status.
+            return Status.FAILED;
+        } else if (statuses.contains(Status.RUNNING)) {
+            // If there are not failed entry and if there is at least one running entry we should return running status.
+            return Status.RUNNING;
+        } else {
+            // If each and every partition has reported success do we report success as the status.
+            int successCount = 0;
+            for (int i = 0; i < statuses.size(); i++) {
+                if (statuses.get(i) == Status.SUCCESS) {
+                    successCount++;
+                }
+            }
+            if (successCount == statuses.size()) {
+                return Status.SUCCESS;
+            }
+        }
+        return status;
+    }
+
+    @Override
+    public synchronized DatasetDirectoryRecord[] getResultPartitionLocations(JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownRecords) throws HyracksDataException {
+        DatasetDirectoryRecord[] newRecords;
+        while ((newRecords = updatedRecords(jobId, rsId, knownRecords)) == null) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        return newRecords;
+    }
+
+    public DatasetDirectoryRecord getDatasetDirectoryRecord(JobId jobId, ResultSetId rsId, int partition) {
+        Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
+        ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+        DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+        return records[partition];
+    }
+
+    /**
+     * Compares the records already known by the client for the given job's result set id with the records that the
+     * dataset directory service knows and if there are any newly discovered records returns a whole array with the
+     * new records filled in.
+     * This method has a very convoluted logic. Here is the explanation of how it works.
+     * If the ordering constraint has to be enforced, the method obtains the first null record in the known records in
+     * the order of the partitions. It always traverses the array in the first to last order!
+     * If known records array or the first element in that array is null in the but the record for that partition now
+     * known to the directory service, the method fills in that record in the array and returns the array back.
+     * However, if the first known null record is not a first element in the array, by induction, all the previous
+     * known records should be known already be known to client and none of the records for the partitions ahead is
+     * known by the client yet. So, we check if the client has reached the end of stream for the partition corresponding
+     * to the record before the first known null record, i.e. the last known non-null record. If not, we just return
+     * null because we cannot expose any new locations until the client reaches end of stream for the last known record.
+     * If the client has reached the end of stream record for the last known non-null record, we check if the next record
+     * is discovered by the dataset directory service and if so, we fill it in the records array and return it back or
+     * send null otherwise.
+     * If the ordering is not required, we are free to return any newly discovered records back, so we just check if
+     * arrays are equal and if they are not we send the entire new updated array.
+     * 
+     * @param jobId
+     *            - Id of the job for which the directory records should be retrieved.
+     * @param rsId
+     *            - Id of the result set for which the directory records should be retrieved.
+     * @param knownRecords
+     *            - An array of directory records that the client is already aware of.
+     * @return
+     *         - Returns null if there aren't any newly discovered partitions enforcing the ordering constraint
+     * @throws HyracksDataException
+     *             TODO(madhusudancs): Think about caching (and still be stateless) instead of this ugly O(n) iterations for
+     *             every check. This already looks very expensive.
+     */
+    private DatasetDirectoryRecord[] updatedRecords(JobId jobId, ResultSetId rsId, DatasetDirectoryRecord[] knownRecords)
+            throws HyracksDataException {
+        Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
+        if (rsMap == null) {
+            return null;
+        }
+
+        ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+        if (resultSetMetaData == null || resultSetMetaData.getRecords() == null) {
+            throw new HyracksDataException("ResultSet locations uninitialized when it is expected to be initialized.");
+        }
+
+        boolean ordered = resultSetMetaData.getOrderedResult();
+        DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+        /* If ordering is required, we should expose the dataset directory records only in the order, otherwise
+         * we can simply check if there are any newly discovered records and send the whole array back if there are.
+         */
+        if (ordered) {
+            // Iterate over the known records and find the last record which is not null.
+            int i = 0;
+            for (i = 0; i < records.length; i++) {
+                if (knownRecords == null) {
+                    if (records[0] != null) {
+                        knownRecords = new DatasetDirectoryRecord[records.length];
+                        knownRecords[0] = records[0];
+                        return knownRecords;
+                    }
+                    return null;
+                }
+                if (knownRecords[i] == null) {
+                    if ((i == 0 || knownRecords[i - 1].hasReachedReadEOS()) && records[i] != null) {
+                        knownRecords[i] = records[i];
+                        return knownRecords;
+                    }
+                    return null;
+                }
+            }
+        } else {
+            if (!Arrays.equals(records, knownRecords)) {
+                return records;
+            }
+        }
+        return null;
+    }
+
+    private class ResultSetMetaData {
+        private final boolean ordered;
+
+        private final DatasetDirectoryRecord[] records;
+
+        public ResultSetMetaData(boolean ordered, DatasetDirectoryRecord[] records) {
+            this.ordered = ordered;
+            this.records = records;
+        }
+
+        public boolean getOrderedResult() {
+            return ordered;
+        }
+
+        public DatasetDirectoryRecord[] getRecords() {
+            return records;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java
new file mode 100644
index 0000000..3ac6acc
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.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.control.cc.work;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class GetDatasetDirectoryServiceInfoWork extends SynchronizableWork {
+    private final ClusterControllerService ccs;
+
+    private final IResultCallback<NetworkAddress> callback;
+
+    public GetDatasetDirectoryServiceInfoWork(ClusterControllerService ccs, IResultCallback<NetworkAddress> callback) {
+        this.ccs = ccs;
+        this.callback = callback;
+    }
+
+    @Override
+    public void doRun() {
+        try {
+            NetworkAddress addr = ccs.getDatasetDirectoryServiceInfo();
+            callback.setValue(addr);
+        } catch (Exception e) {
+            callback.setException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
index 2f23a2c..a787b9f 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
@@ -39,7 +39,8 @@
         Map<String, NodeControllerInfo> result = new LinkedHashMap<String, NodeControllerInfo>();
         Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
         for (Map.Entry<String, NodeControllerState> e : nodeMap.entrySet()) {
-            result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE, e.getValue().getDataPort()));
+            result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE, e.getValue().getDataPort(), e
+                    .getValue().getDatasetPort()));
         }
         callback.setValue(result);
     }
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultPartitionLocationsWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
new file mode 100644
index 0000000..fd1d418
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultPartitionLocationsWork.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.control.cc.work;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class GetResultPartitionLocationsWork extends SynchronizableWork {
+    private final ClusterControllerService ccs;
+
+    private final JobId jobId;
+
+    private final ResultSetId rsId;
+
+    private final DatasetDirectoryRecord[] knownRecords;
+
+    private final IResultCallback<DatasetDirectoryRecord[]> callback;
+
+    public GetResultPartitionLocationsWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownRecords, IResultCallback<DatasetDirectoryRecord[]> callback) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.rsId = rsId;
+        this.knownRecords = knownRecords;
+        this.callback = callback;
+    }
+
+    @Override
+    public void doRun() {
+        final IDatasetDirectoryService dds = ccs.getDatasetDirectoryService();
+        ccs.getExecutor().execute(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    DatasetDirectoryRecord[] partitionLocations = dds.getResultPartitionLocations(jobId, rsId,
+                            knownRecords);
+                    callback.setValue(partitionLocations);
+                } catch (HyracksDataException e) {
+                    callback.setException(e);
+                }
+            }
+        });
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java
new file mode 100644
index 0000000..d2dadf5
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java
@@ -0,0 +1,56 @@
+/*
+ * 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.control.cc.work;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class GetResultStatusWork extends SynchronizableWork {
+    private final ClusterControllerService ccs;
+
+    private final JobId jobId;
+
+    private final ResultSetId rsId;
+
+    private final IResultCallback<Status> callback;
+
+    public GetResultStatusWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+            IResultCallback<Status> callback) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.rsId = rsId;
+        this.callback = callback;
+    }
+
+    @Override
+    public void doRun() {
+        try {
+            Status status = ccs.getDatasetDirectoryService().getResultStatus(jobId, rsId);
+            callback.setValue(status);
+        } catch (HyracksDataException e) {
+            callback.setException(e);
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "JobId@" + jobId + " ResultSetId@" + rsId;
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
index b6a33cd..b062d33 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
@@ -16,6 +16,7 @@
 
 import java.util.EnumSet;
 
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGenerator;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
new file mode 100644
index 0000000..f86e924
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterResultPartitionLocationWork.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.control.cc.work;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class RegisterResultPartitionLocationWork extends AbstractWork {
+    private final ClusterControllerService ccs;
+
+    private final JobId jobId;
+
+    private final ResultSetId rsId;
+
+    private final boolean orderedResult;
+
+    private final int partition;
+
+    private final int nPartitions;
+
+    private final NetworkAddress networkAddress;
+
+    public RegisterResultPartitionLocationWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+            boolean orderedResult, int partition, int nPartitions, NetworkAddress networkAddress) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.rsId = rsId;
+        this.orderedResult = orderedResult;
+        this.partition = partition;
+        this.nPartitions = nPartitions;
+        this.networkAddress = networkAddress;
+    }
+
+    @Override
+    public void run() {
+        ccs.getDatasetDirectoryService().registerResultPartitionLocation(jobId, rsId, orderedResult, partition,
+                nPartitions, networkAddress);
+    }
+
+    @Override
+    public String toString() {
+        return "JobId@" + jobId + " ResultSetId@" + rsId + " Partition@" + partition + " NPartitions@" + nPartitions
+                + " ResultPartitionLocation@" + networkAddress + " OrderedResult@" + orderedResult;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionFailureWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionFailureWork.java
new file mode 100644
index 0000000..4aea41e
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionFailureWork.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.control.cc.work;
+
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class ReportResultPartitionFailureWork extends AbstractWork {
+    private final ClusterControllerService ccs;
+
+    private final JobId jobId;
+
+    private final ResultSetId rsId;
+
+    private final int partition;
+
+    public ReportResultPartitionFailureWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId, int partition) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.rsId = rsId;
+        this.partition = partition;
+    }
+
+    @Override
+    public void run() {
+        ccs.getDatasetDirectoryService().reportResultPartitionFailure(jobId, rsId, partition);
+    }
+
+    @Override
+    public String toString() {
+        return "JobId@" + jobId + " ResultSetId@" + rsId + " Partition@" + partition;
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java
new file mode 100644
index 0000000..313b730
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java
@@ -0,0 +1,48 @@
+/*
+ * 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.control.cc.work;
+
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class ReportResultPartitionWriteCompletionWork extends AbstractWork {
+    private final ClusterControllerService ccs;
+
+    private final JobId jobId;
+
+    private final ResultSetId rsId;
+
+    private final int partition;
+
+    public ReportResultPartitionWriteCompletionWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+            int partition) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.rsId = rsId;
+        this.partition = partition;
+    }
+
+    @Override
+    public void run() {
+        ccs.getDatasetDirectoryService().reportResultPartitionWriteCompletion(jobId, rsId, partition);
+    }
+
+    @Override
+    public String toString() {
+        return "JobId@" + jobId + " ResultSetId@" + rsId + " Partition@" + partition;
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js b/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js
index ff9d8a0..8e94269 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js
@@ -58,6 +58,10 @@
         var netPayloadBytesWritten = result['net-payload-bytes-written'];
         var netSignalingBytesRead = result['net-signaling-bytes-read'];
         var netSignalingBytesWritten = result['net-signaling-bytes-written'];
+        var datasetNetPayloadBytesRead = result['dataset-net-payload-bytes-read'];
+        var datasetNetPayloadBytesWritten = result['dataset-net-payload-bytes-written'];
+        var datasetNetSignalingBytesRead = result['dataset-net-signaling-bytes-read'];
+        var datasetNetSignalingBytesWritten = result['dataset-net-signaling-bytes-written'];
         var ipcMessagesSent = result['ipc-messages-sent'];
         var ipcMessageBytesSent = result['ipc-message-bytes-sent'];
         var ipcMessagesReceived = result['ipc-messages-received'];
@@ -117,8 +121,12 @@
             }
             if (i < sysLoad.length - 1) {
                 netPayloadReadBWArray.push([ i, computeRate(netPayloadBytesRead, rrdPtr) ]);
+                netPayloadReadBWArray.push([ i, computeRate(datasetNetPayloadBytesRead, rrdPtr) ]);
                 netPayloadWriteBWArray.push([ i, computeRate(netPayloadBytesWritten, rrdPtr) ]);
+                netPayloadWriteBWArray.push([ i, computeRate(datasetNetPayloadBytesWritten, rrdPtr) ]);
                 netSignalingReadBWArray.push([ i, computeRate(netSignalingBytesRead, rrdPtr) ]);
+                netSignalingReadBWArray.push([ i, computeRate(datasetNetSignalingBytesRead, rrdPtr) ]);
+                netSignalingWriteBWArray.push([ i, computeRate(netSignalingBytesWritten, rrdPtr) ]);
                 netSignalingWriteBWArray.push([ i, computeRate(netSignalingBytesWritten, rrdPtr) ]);
                 ipcMessageSendRateArray.push([ i, computeRate(ipcMessagesSent, rrdPtr) ]);
                 ipcMessageBytesSendRateArray.push([ i, computeRate(ipcMessageBytesSent, rrdPtr) ]);
@@ -229,4 +237,4 @@
     }
 
     fetchData();
-});
\ No newline at end of file
+});
diff --git a/hyracks/hyracks-control/hyracks-control-common/pom.xml b/hyracks/hyracks-control/hyracks-control-common/pom.xml
index 096f2e1..ce1298e 100644
--- a/hyracks/hyracks-control/hyracks-control-common/pom.xml
+++ b/hyracks/hyracks-control/hyracks-control-common/pom.xml
@@ -18,8 +18,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
index 0c5bb2f..55e4479 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
@@ -16,7 +16,9 @@
 
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
 import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
@@ -46,6 +48,13 @@
 
     public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception;
 
+    public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
+            int nPartitions, NetworkAddress networkAddress) throws Exception;
+
+    public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws Exception;
+
+    public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws Exception;
+
     public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception;
 
     public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception;
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index 167eb4b..5071bc9 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -37,6 +37,9 @@
     @Option(name = "-data-ip-address", usage = "IP Address to bind data listener", required = true)
     public String dataIPAddress;
 
+    @Option(name = "-result-ip-address", usage = "IP Address to bind dataset result distribution listener", required = true)
+    public String datasetIPAddress;
+
     @Option(name = "-iodevices", usage = "Comma separated list of IO Device mount points (default: One device in default temp folder)", required = false)
     public String ioDevices = System.getProperty("java.io.tmpdir");
 
@@ -55,6 +58,9 @@
     @Option(name = "-max-memory", usage = "Maximum memory usable at this Node Controller in bytes (default: -1 auto)")
     public int maxMemory = -1;
 
+    @Option(name = "-result-manager-memory", usage = "Memory usable for result caching at this Node Controller in bytes (default: -1 auto)")
+    public int resultManagerMemory = -1;
+
     public void toCommandLine(List<String> cList) {
         cList.add("-cc-host");
         cList.add(ccHost);
@@ -66,6 +72,7 @@
         cList.add(nodeId);
         cList.add("-data-ip-address");
         cList.add(dataIPAddress);
+        cList.add(datasetIPAddress);
         cList.add("-iodevices");
         cList.add(ioDevices);
         cList.add("-dcache-client-servers");
@@ -80,5 +87,7 @@
         cList.add(String.valueOf(nNetThreads));
         cList.add("-max-memory");
         cList.add(String.valueOf(maxMemory));
+        cList.add("-result-manager-memory");
+        cList.add(String.valueOf(resultManagerMemory));
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
index 91cfecf..a897602 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
@@ -33,6 +33,8 @@
 
     private final NetworkAddress dataPort;
 
+    private final NetworkAddress datasetPort;
+
     private final String osName;
 
     private final String arch;
@@ -60,13 +62,14 @@
     private final HeartbeatSchema hbSchema;
 
     public NodeRegistration(InetSocketAddress ncAddress, String nodeId, NCConfig ncConfig, NetworkAddress dataPort,
-            String osName, String arch, String osVersion, int nProcessors, String vmName, String vmVersion,
-            String vmVendor, String classpath, String libraryPath, String bootClasspath, List<String> inputArguments,
-            Map<String, String> systemProperties, HeartbeatSchema hbSchema) {
+            NetworkAddress datasetPort, String osName, String arch, String osVersion, int nProcessors, String vmName,
+            String vmVersion, String vmVendor, String classpath, String libraryPath, String bootClasspath,
+            List<String> inputArguments, Map<String, String> systemProperties, HeartbeatSchema hbSchema) {
         this.ncAddress = ncAddress;
         this.nodeId = nodeId;
         this.ncConfig = ncConfig;
         this.dataPort = dataPort;
+        this.datasetPort = datasetPort;
         this.osName = osName;
         this.arch = arch;
         this.osVersion = osVersion;
@@ -98,6 +101,10 @@
         return dataPort;
     }
 
+    public NetworkAddress getDatasetPort() {
+        return datasetPort;
+    }
+
     public String getOSName() {
         return osName;
     }
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
index 1dba3bc..663c68a 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
@@ -37,6 +37,10 @@
     public long netPayloadBytesWritten;
     public long netSignalingBytesRead;
     public long netSignalingBytesWritten;
+    public long datasetNetPayloadBytesRead;
+    public long datasetNetPayloadBytesWritten;
+    public long datasetNetSignalingBytesRead;
+    public long datasetNetSignalingBytesWritten;
     public long ipcMessagesSent;
     public long ipcMessageBytesSent;
     public long ipcMessagesReceived;
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
index 557a8cb..b506b12 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
@@ -36,6 +36,7 @@
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -68,6 +69,9 @@
         REPORT_PROFILE,
         REGISTER_PARTITION_PROVIDER,
         REGISTER_PARTITION_REQUEST,
+        REGISTER_RESULT_PARTITION_LOCATION,
+        REPORT_RESULT_PARTITION_WRITE_COMPLETION,
+        REPORT_RESULT_PARTITION_FAILURE,
         APPLICATION_STATE_CHANGE_RESPONSE,
 
         NODE_REGISTRATION_RESULT,
@@ -438,6 +442,127 @@
         }
     }
 
+    public static class RegisterResultPartitionLocationFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+
+        private final ResultSetId rsId;
+
+        private final boolean orderedResult;
+
+        private final int partition;
+
+        private final int nPartitions;
+
+        private NetworkAddress networkAddress;
+
+        public RegisterResultPartitionLocationFunction(JobId jobId, ResultSetId rsId, boolean orderedResult,
+                int partition, int nPartitions, NetworkAddress networkAddress) {
+            this.jobId = jobId;
+            this.rsId = rsId;
+            this.orderedResult = orderedResult;
+            this.partition = partition;
+            this.nPartitions = nPartitions;
+            this.networkAddress = networkAddress;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.REGISTER_RESULT_PARTITION_LOCATION;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public ResultSetId getResultSetId() {
+            return rsId;
+        }
+
+        public boolean getOrderedResult() {
+            return orderedResult;
+        }
+
+        public int getPartition() {
+            return partition;
+        }
+
+        public int getNPartitions() {
+            return nPartitions;
+        }
+
+        public NetworkAddress getNetworkAddress() {
+            return networkAddress;
+        }
+    }
+
+    public static class ReportResultPartitionWriteCompletionFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+
+        private final ResultSetId rsId;
+
+        private final int partition;
+
+        public ReportResultPartitionWriteCompletionFunction(JobId jobId, ResultSetId rsId, int partition) {
+            this.jobId = jobId;
+            this.rsId = rsId;
+            this.partition = partition;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.REPORT_RESULT_PARTITION_WRITE_COMPLETION;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public ResultSetId getResultSetId() {
+            return rsId;
+        }
+
+        public int getPartition() {
+            return partition;
+        }
+    }
+
+    public static class ReportResultPartitionFailureFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+
+        private final ResultSetId rsId;
+
+        private final int partition;
+
+        public ReportResultPartitionFailureFunction(JobId jobId, ResultSetId rsId, int partition) {
+            this.jobId = jobId;
+            this.rsId = rsId;
+            this.partition = partition;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.REPORT_RESULT_PARTITION_FAILURE;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public ResultSetId getResultSetId() {
+            return rsId;
+        }
+
+        public int getPartition() {
+            return partition;
+        }
+    }
+
     public static class ApplicationStateChangeResponseFunction extends Function {
         private static final long serialVersionUID = 1L;
 
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
index bbaab4e..091a5d2 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
@@ -16,7 +16,9 @@
 
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
 import edu.uci.ics.hyracks.control.common.base.IClusterController;
@@ -95,6 +97,28 @@
     }
 
     @Override
+    public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
+            int nPartitions, NetworkAddress networkAddress) throws Exception {
+        CCNCFunctions.RegisterResultPartitionLocationFunction fn = new CCNCFunctions.RegisterResultPartitionLocationFunction(
+                jobId, rsId, orderedResult, partition, nPartitions, networkAddress);
+        ipcHandle.send(-1, fn, null);
+    }
+
+    @Override
+    public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws Exception {
+        CCNCFunctions.ReportResultPartitionWriteCompletionFunction fn = new CCNCFunctions.ReportResultPartitionWriteCompletionFunction(
+                jobId, rsId, partition);
+        ipcHandle.send(-1, fn, null);
+    }
+
+    @Override
+    public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws Exception {
+        CCNCFunctions.ReportResultPartitionFailureFunction fn = new CCNCFunctions.ReportResultPartitionFailureFunction(
+                jobId, rsId, partition);
+        ipcHandle.send(-1, fn, null);
+    }
+
+    @Override
     public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception {
         CCNCFunctions.ApplicationStateChangeResponseFunction fn = new CCNCFunctions.ApplicationStateChangeResponseFunction(
                 nodeId, appName, status);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/pom.xml b/hyracks/hyracks-control/hyracks-control-nc/pom.xml
index 9ecc083..c44cec9 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/pom.xml
+++ b/hyracks/hyracks-control/hyracks-control-nc/pom.xml
@@ -15,8 +15,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
@@ -40,6 +41,11 @@
   		<artifactId>hyracks-net</artifactId>
   		<version>0.2.3-SNAPSHOT</version>
   	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-comm</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  	</dependency>
   </dependencies>
   <reporting>
     <plugins>
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 0195143..8a36dac 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -45,6 +45,7 @@
 
 import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
 import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
@@ -61,7 +62,9 @@
 import edu.uci.ics.hyracks.control.common.work.FutureValue;
 import edu.uci.ics.hyracks.control.common.work.WorkQueue;
 import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
+import edu.uci.ics.hyracks.control.nc.dataset.DatasetPartitionManager;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.net.DatasetNetworkManager;
 import edu.uci.ics.hyracks.control.nc.net.NetworkManager;
 import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
 import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
@@ -94,6 +97,10 @@
 
     private final NetworkManager netManager;
 
+    private final IDatasetPartitionManager datasetPartitionManager;
+
+    private final DatasetNetworkManager datasetNetworkManager;
+
     private final WorkQueue queue;
 
     private final Timer timer;
@@ -140,7 +147,11 @@
             throw new Exception("id not set");
         }
         partitionManager = new PartitionManager(this);
-        netManager = new NetworkManager(getIpAddress(ncConfig), partitionManager, ncConfig.nNetThreads);
+        netManager = new NetworkManager(getIpAddress(ncConfig.dataIPAddress), partitionManager, ncConfig.nNetThreads);
+
+        datasetPartitionManager = new DatasetPartitionManager(this, executor, ncConfig.resultManagerMemory);
+        datasetNetworkManager = new DatasetNetworkManager(getIpAddress(ncConfig.datasetIPAddress),
+                datasetPartitionManager, ncConfig.nNetThreads);
 
         queue = new WorkQueue();
         jobletMap = new Hashtable<JobId, Joblet>();
@@ -205,6 +216,7 @@
         LOGGER.log(Level.INFO, "Starting NodeControllerService");
         ipc.start();
         netManager.start();
+        datasetNetworkManager.start();
         IIPCHandle ccIPCHandle = ipc.getHandle(new InetSocketAddress(ncConfig.ccHost, ncConfig.ccPort));
         this.ccs = new ClusterControllerRemoteProxy(ccIPCHandle);
         HeartbeatSchema.GarbageCollectorInfo[] gcInfos = new HeartbeatSchema.GarbageCollectorInfo[gcMXBeans.size()];
@@ -213,10 +225,11 @@
         }
         HeartbeatSchema hbSchema = new HeartbeatSchema(gcInfos);
         ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig, netManager.getNetworkAddress(),
-                osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), osMXBean.getAvailableProcessors(),
-                runtimeMXBean.getVmName(), runtimeMXBean.getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean
-                        .getClassPath(), runtimeMXBean.getLibraryPath(), runtimeMXBean.getBootClassPath(),
-                runtimeMXBean.getInputArguments(), runtimeMXBean.getSystemProperties(), hbSchema));
+                datasetNetworkManager.getNetworkAddress(), osMXBean.getName(), osMXBean.getArch(), osMXBean
+                        .getVersion(), osMXBean.getAvailableProcessors(), runtimeMXBean.getVmName(), runtimeMXBean
+                        .getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean.getClassPath(), runtimeMXBean
+                        .getLibraryPath(), runtimeMXBean.getBootClassPath(), runtimeMXBean.getInputArguments(),
+                runtimeMXBean.getSystemProperties(), hbSchema));
 
         synchronized (this) {
             while (registrationPending) {
@@ -247,8 +260,10 @@
         LOGGER.log(Level.INFO, "Stopping NodeControllerService");
         executor.shutdownNow();
         partitionManager.close();
+        datasetPartitionManager.close();
         heartbeatTask.cancel();
         netManager.stop();
+        datasetNetworkManager.stop();
         queue.stop();
         LOGGER.log(Level.INFO, "Stopped NodeControllerService");
     }
@@ -273,6 +288,10 @@
         return netManager;
     }
 
+    public DatasetNetworkManager getDatasetNetworkManager() {
+        return datasetNetworkManager;
+    }
+
     public PartitionManager getPartitionManager() {
         return partitionManager;
     }
@@ -297,8 +316,7 @@
         return queue;
     }
 
-    private static InetAddress getIpAddress(NCConfig ncConfig) throws Exception {
-        String ipaddrStr = ncConfig.dataIPAddress;
+    private static InetAddress getIpAddress(String ipaddrStr) throws Exception {
         ipaddrStr = ipaddrStr.trim();
         Pattern pattern = Pattern.compile("(\\d{1,3})\\.(\\d{1,3})\\.(\\d{1,3})\\.(\\d{1,3})");
         Matcher m = pattern.matcher(ipaddrStr);
@@ -355,6 +373,12 @@
             hbData.netSignalingBytesRead = netPC.getSignalingBytesRead();
             hbData.netSignalingBytesWritten = netPC.getSignalingBytesWritten();
 
+            MuxDemuxPerformanceCounters datasetNetPC = datasetNetworkManager.getPerformanceCounters();
+            hbData.datasetNetPayloadBytesRead = datasetNetPC.getPayloadBytesRead();
+            hbData.datasetNetPayloadBytesWritten = datasetNetPC.getPayloadBytesWritten();
+            hbData.datasetNetSignalingBytesRead = datasetNetPC.getSignalingBytesRead();
+            hbData.datasetNetSignalingBytesWritten = datasetNetPC.getSignalingBytesWritten();
+
             IPCPerformanceCounters ipcPC = ipc.getPerformanceCounters();
             hbData.ipcMessagesSent = ipcPC.getMessageSentCount();
             hbData.ipcMessageBytesSent = ipcPC.getMessageBytesSent();
@@ -459,6 +483,10 @@
         }
     }
 
+    public IDatasetPartitionManager getDatasetPartitionManager() {
+        return datasetPartitionManager;
+    }
+
     public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception {
         ccs.sendApplicationMessageToCC(data, appName, nodeId);
     }
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index eba3ec9..5a3e9dd 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -34,6 +34,7 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -348,6 +349,11 @@
     }
 
     @Override
+    public IDatasetPartitionManager getDatasetPartitionManager() {
+        return ncs.getDatasetPartitionManager();
+    }
+
+    @Override
     public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception {
         this.ncs.sendApplicationMessageToCC(message, this.getJobletContext().getApplicationContext()
                 .getApplicationName(), nodeId);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java
new file mode 100644
index 0000000..cecd677
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java
@@ -0,0 +1,237 @@
+/*
+ * 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.control.nc.dataset;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionWriter;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
+
+public class DatasetMemoryManager {
+    private final Set<Page> availPages;
+
+    private final LeastRecentlyUsedList leastRecentlyUsedList;
+
+    private final Map<ResultSetPartitionId, PartitionNode> resultPartitionNodesMap;
+
+    private final static int FRAME_SIZE = 32768;
+
+    public DatasetMemoryManager(int availableMemory) {
+        availPages = new HashSet<Page>();
+
+        // Atleast have one page for temporarily storing the results.
+        if (availableMemory <= 0)
+            availableMemory = FRAME_SIZE;
+
+        while (availableMemory >= FRAME_SIZE) {
+            /* TODO(madhusudancs): Should we have some way of accounting this memory usage by using Hyrack's allocateFrame()
+             * instead of direct ByteBuffer.allocate()?
+             */
+            availPages.add(new Page(ByteBuffer.allocate(FRAME_SIZE)));
+            availableMemory -= FRAME_SIZE;
+        }
+
+        leastRecentlyUsedList = new LeastRecentlyUsedList();
+        resultPartitionNodesMap = new HashMap<ResultSetPartitionId, PartitionNode>();
+    }
+
+    public Page requestPage(ResultSetPartitionId resultSetPartitionId, IDatasetPartitionWriter dpw)
+            throws OutOfMemoryError, HyracksDataException {
+        Page page;
+        if (availPages.isEmpty()) {
+            page = evictPage();
+        } else {
+            page = getAvailablePage();
+        }
+
+        page.clear();
+
+        /*
+         * It is extremely important to update the reference after obtaining the page because, in the cases where memory
+         * manager is allocated only one page of memory, the front of the LRU list should not be created by the
+         * update reference call before a page is pushed on to the element of the LRU list. So we first obtain the page,
+         * then make a updateReference call which in turn creates a new node in the LRU list and then add the page to it.
+         */
+        PartitionNode pn = updateReference(resultSetPartitionId, dpw);
+        pn.add(page);
+        return page;
+    }
+
+    public void pageReferenced(ResultSetPartitionId resultSetPartitionId) {
+        // When a page is referenced the dataset partition writer should already be known, so we pass null.
+        updateReference(resultSetPartitionId, null);
+    }
+
+    public int getPageSize() {
+        return FRAME_SIZE;
+    }
+
+    protected void insertPartitionNode(ResultSetPartitionId resultSetPartitionId, PartitionNode pn) {
+        leastRecentlyUsedList.add(pn);
+        resultPartitionNodesMap.put(resultSetPartitionId, pn);
+    }
+
+    protected synchronized PartitionNode updateReference(ResultSetPartitionId resultSetPartitionId,
+            IDatasetPartitionWriter dpw) {
+        PartitionNode pn = null;
+
+        if (!resultPartitionNodesMap.containsKey(resultSetPartitionId)) {
+            if (dpw != null) {
+                pn = new PartitionNode(resultSetPartitionId, dpw);
+                insertPartitionNode(resultSetPartitionId, pn);
+            }
+            return pn;
+        }
+        pn = resultPartitionNodesMap.get(resultSetPartitionId);
+        leastRecentlyUsedList.remove(pn);
+        insertPartitionNode(resultSetPartitionId, pn);
+
+        return pn;
+    }
+
+    protected synchronized Page evictPage() throws HyracksDataException {
+        PartitionNode pn = leastRecentlyUsedList.getFirst();
+        IDatasetPartitionWriter dpw = pn.getDatasetPartitionWriter();
+        Page page = dpw.returnPage();
+
+        /* If the partition holding the pages breaks the contract by not returning the page or it has no page, just take
+         * away all the pages allocated to it and add to the available pages set.
+         */
+        if (page == null) {
+            availPages.addAll(pn);
+            pn.clear();
+            resultPartitionNodesMap.remove(pn.getResultSetPartitionId());
+            leastRecentlyUsedList.remove(pn);
+
+            /* Based on the assumption that if the dataset partition writer returned a null page, it should be lying about
+             * the number of pages it holds in which case we just evict all the pages it holds and should thus be able to
+             * add all those pages to available set and we have at least one page to allocate back.
+             */
+            page = getAvailablePage();
+        } else {
+            pn.remove(page);
+
+            // If the partition no more holds any pages, remove it from the linked list and the hash map.
+            if (pn.isEmpty()) {
+                resultPartitionNodesMap.remove(pn.getResultSetPartitionId());
+                leastRecentlyUsedList.remove(pn);
+            }
+        }
+
+        return page;
+    }
+
+    protected synchronized Page getAvailablePage() {
+        Iterator<Page> iter = availPages.iterator();
+        Page page = iter.next();
+        iter.remove();
+        return page;
+    }
+
+    private class LeastRecentlyUsedList {
+        private PartitionNode head;
+
+        private PartitionNode tail;
+
+        public LeastRecentlyUsedList() {
+            head = null;
+            tail = null;
+        }
+
+        public void add(PartitionNode node) {
+            if (head == null) {
+                head = tail = node;
+                return;
+            }
+            tail.setNext(node);
+            node.setPrev(tail);
+            tail = node;
+        }
+
+        public void remove(PartitionNode node) {
+            if ((node == head) && (node == tail)) {
+                head = tail = null;
+                return;
+            } else if (node == head) {
+                head = head.getNext();
+                head.setPrev(null);
+                return;
+            } else if (node == tail) {
+                tail = tail.getPrev();
+                tail.setNext(null);
+                return;
+            } else {
+                PartitionNode prev = node.getPrev();
+                PartitionNode next = node.getNext();
+                prev.setNext(next);
+                next.setPrev(prev);
+            }
+        }
+
+        public PartitionNode getFirst() {
+            return head;
+        }
+    }
+
+    private class PartitionNode extends HashSet<Page> {
+        private static final long serialVersionUID = 1L;
+
+        private final ResultSetPartitionId resultSetPartitionId;
+
+        private final IDatasetPartitionWriter datasetPartitionWriter;
+
+        private PartitionNode prev;
+
+        private PartitionNode next;
+
+        public PartitionNode(ResultSetPartitionId resultSetPartitionId, IDatasetPartitionWriter datasetPartitionWriter) {
+            this.resultSetPartitionId = resultSetPartitionId;
+            this.datasetPartitionWriter = datasetPartitionWriter;
+            prev = null;
+            next = null;
+        }
+
+        public ResultSetPartitionId getResultSetPartitionId() {
+            return resultSetPartitionId;
+        }
+
+        public IDatasetPartitionWriter getDatasetPartitionWriter() {
+            return datasetPartitionWriter;
+        }
+
+        public void setPrev(PartitionNode node) {
+            prev = node;
+        }
+
+        public PartitionNode getPrev() {
+            return prev;
+        }
+
+        public void setNext(PartitionNode node) {
+            next = node;
+        }
+
+        public PartitionNode getNext() {
+            return next;
+        }
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java
new file mode 100644
index 0000000..1cad54b
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java
@@ -0,0 +1,124 @@
+/*
+ * 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.control.nc.dataset;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
+
+public class DatasetPartitionManager implements IDatasetPartitionManager {
+    private final NodeControllerService ncs;
+
+    private final Executor executor;
+
+    private final Map<JobId, ResultState[]> partitionResultStateMap;
+
+    private final DefaultDeallocatableRegistry deallocatableRegistry;
+
+    private final IWorkspaceFileFactory fileFactory;
+
+    private final DatasetMemoryManager datasetMemoryManager;
+
+    public DatasetPartitionManager(NodeControllerService ncs, Executor executor, int availableMemory) {
+        this.ncs = ncs;
+        this.executor = executor;
+        partitionResultStateMap = new HashMap<JobId, ResultState[]>();
+        deallocatableRegistry = new DefaultDeallocatableRegistry();
+        fileFactory = new WorkspaceFileFactory(deallocatableRegistry, (IOManager) ncs.getRootContext().getIOManager());
+        datasetMemoryManager = new DatasetMemoryManager(availableMemory);
+    }
+
+    @Override
+    public IFrameWriter createDatasetPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
+            int partition, int nPartitions) throws HyracksException {
+        DatasetPartitionWriter dpw = null;
+        JobId jobId = ctx.getJobletContext().getJobId();
+        try {
+            ncs.getClusterController().registerResultPartitionLocation(jobId, rsId, orderedResult, partition,
+                    nPartitions, ncs.getDatasetNetworkManager().getNetworkAddress());
+            dpw = new DatasetPartitionWriter(ctx, this, jobId, rsId, partition, datasetMemoryManager);
+
+            ResultState[] resultStates = partitionResultStateMap.get(jobId);
+            if (resultStates == null) {
+                resultStates = new ResultState[nPartitions];
+                partitionResultStateMap.put(jobId, resultStates);
+            }
+            resultStates[partition] = dpw.getResultState();
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+
+        return dpw;
+    }
+
+    @Override
+    public void reportPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws HyracksException {
+        try {
+            ncs.getClusterController().reportResultPartitionWriteCompletion(jobId, rsId, partition);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    @Override
+    public void reportPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws HyracksException {
+        try {
+            ncs.getClusterController().reportResultPartitionFailure(jobId, rsId, partition);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    @Override
+    public void initializeDatasetPartitionReader(JobId jobId, int partition, IFrameWriter writer)
+            throws HyracksException {
+        ResultState[] resultStates = partitionResultStateMap.get(jobId);
+
+        if (resultStates == null) {
+            throw new HyracksException("Unknown JobId " + jobId);
+        }
+
+        ResultState resultState = resultStates[partition];
+        if (resultState == null) {
+            throw new HyracksException("No DatasetPartitionWriter for partition " + partition);
+        }
+
+        IDatasetPartitionReader dpr = new DatasetPartitionReader(datasetMemoryManager, executor, resultState);
+        dpr.writeTo(writer);
+    }
+
+    @Override
+    public IWorkspaceFileFactory getFileFactory() {
+        return fileFactory;
+    }
+
+    @Override
+    public void close() {
+        deallocatableRegistry.close();
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java
new file mode 100644
index 0000000..296c502
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java
@@ -0,0 +1,122 @@
+/*
+ * 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.control.nc.dataset;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.Executor;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionReader;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.IFileHandle;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
+
+public class DatasetPartitionReader implements IDatasetPartitionReader {
+    private static final Logger LOGGER = Logger.getLogger(DatasetPartitionReader.class.getName());
+
+    private final DatasetMemoryManager datasetMemoryManager;
+
+    private final Executor executor;
+
+    private final ResultState resultState;
+
+    private IFileHandle fileHandle;
+
+    public DatasetPartitionReader(DatasetMemoryManager datasetMemoryManager, Executor executor, ResultState resultState) {
+        this.datasetMemoryManager = datasetMemoryManager;
+        this.executor = executor;
+        this.resultState = resultState;
+    }
+
+    private long read(long offset, ByteBuffer buffer) throws HyracksDataException {
+        long readSize = 0;
+        synchronized (resultState) {
+            while (offset >= resultState.getSize() && !resultState.getEOS()) {
+                try {
+                    resultState.wait();
+                } catch (InterruptedException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        }
+
+        if (offset >= resultState.getSize() && resultState.getEOS()) {
+            return readSize;
+        }
+
+        if (offset < resultState.getPersistentSize()) {
+            readSize = resultState.getIOManager().syncRead(fileHandle, offset, buffer);
+        }
+
+        if (readSize < buffer.capacity()) {
+            long localPageOffset = offset - resultState.getPersistentSize();
+            int localPageIndex = (int) (localPageOffset / datasetMemoryManager.getPageSize());
+            int pageOffset = (int) (localPageOffset % datasetMemoryManager.getPageSize());
+            Page page = resultState.getPage(localPageIndex);
+            readSize += buffer.remaining();
+            buffer.put(page.getBuffer().array(), pageOffset, buffer.remaining());
+        }
+
+        datasetMemoryManager.pageReferenced(resultState.getResultSetPartitionId());
+        return readSize;
+    }
+
+    @Override
+    public void writeTo(final IFrameWriter writer) {
+        executor.execute(new Runnable() {
+            @Override
+            public void run() {
+                NetworkOutputChannel channel = (NetworkOutputChannel) writer;
+                channel.setFrameSize(resultState.getFrameSize());
+                try {
+                    fileHandle = resultState.getIOManager().open(resultState.getValidFileReference(),
+                            IIOManager.FileReadWriteMode.READ_ONLY, IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+                    channel.open();
+                    try {
+                        long offset = 0;
+                        ByteBuffer buffer = ByteBuffer.allocate(resultState.getFrameSize());
+                        while (true) {
+                            buffer.clear();
+                            long size = read(offset, buffer);
+                            if (size <= 0) {
+                                break;
+                            } else if (size < buffer.limit()) {
+                                throw new HyracksDataException("Premature end of file - readSize: " + size
+                                        + " buffer limit: " + buffer.limit());
+                            }
+                            offset += size;
+                            buffer.flip();
+                            channel.nextFrame(buffer);
+                        }
+                    } finally {
+                        channel.close();
+                        resultState.getIOManager().close(fileHandle);
+                    }
+                } catch (InterruptedException e) {
+                    throw new RuntimeException(e);
+                } catch (HyracksDataException e) {
+                    throw new RuntimeException(e);
+                }
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("result reading successful(" + resultState.getResultSetPartitionId() + ")");
+                }
+            }
+        });
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
new file mode 100644
index 0000000..f6ae540
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
@@ -0,0 +1,148 @@
+/*
+ * 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.control.nc.dataset;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionWriter;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IFileHandle;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
+
+public class DatasetPartitionWriter implements IDatasetPartitionWriter {
+    private static final Logger LOGGER = Logger.getLogger(DatasetPartitionWriter.class.getName());
+
+    private static final String FILE_PREFIX = "result_";
+
+    private final IDatasetPartitionManager manager;
+
+    private final JobId jobId;
+
+    private final ResultSetId resultSetId;
+
+    private final int partition;
+
+    private final DatasetMemoryManager datasetMemoryManager;
+
+    private final ResultSetPartitionId resultSetPartitionId;
+
+    private final ResultState resultState;
+
+    private IFileHandle fileHandle;
+
+    public DatasetPartitionWriter(IHyracksTaskContext ctx, IDatasetPartitionManager manager, JobId jobId,
+            ResultSetId rsId, int partition, DatasetMemoryManager datasetMemoryManager) {
+        this.manager = manager;
+        this.jobId = jobId;
+        this.resultSetId = rsId;
+        this.partition = partition;
+        this.datasetMemoryManager = datasetMemoryManager;
+
+        resultSetPartitionId = new ResultSetPartitionId(jobId, rsId, partition);
+        resultState = new ResultState(resultSetPartitionId, ctx.getIOManager(), ctx.getFrameSize());
+    }
+
+    public ResultState getResultState() {
+        return resultState;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("open(" + partition + ")");
+        }
+        String fName = FILE_PREFIX + String.valueOf(partition);
+        FileReference fRef = manager.getFileFactory().createUnmanagedWorkspaceFile(fName);
+        fileHandle = resultState.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+        resultState.init(fRef);
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        int srcOffset = 0;
+        Page destPage = resultState.getLastPage();
+
+        while (srcOffset < buffer.limit()) {
+            if ((destPage == null) || (destPage.getBuffer().remaining() <= 0)) {
+                destPage = datasetMemoryManager.requestPage(resultSetPartitionId, this);
+                resultState.addPage(destPage);
+            }
+            int srcLength = Math.min(buffer.limit() - srcOffset, destPage.getBuffer().remaining());
+            destPage.getBuffer().put(buffer.array(), srcOffset, srcLength);
+            srcOffset += srcLength;
+            resultState.incrementSize(srcLength);
+        }
+
+        synchronized (resultState) {
+            resultState.notifyAll();
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        try {
+            manager.reportPartitionFailure(jobId, resultSetId, partition);
+        } catch (HyracksException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("close(" + partition + ")");
+        }
+
+        try {
+            synchronized (resultState) {
+                resultState.setEOS(true);
+                resultState.notifyAll();
+            }
+            manager.reportPartitionWriteCompletion(jobId, resultSetId, partition);
+        } catch (HyracksException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public Page returnPage() throws HyracksDataException {
+        Page page = resultState.removePage(0);
+
+        IIOManager ioManager = resultState.getIOManager();
+
+        // If we do not have any pages to be given back close the write channel since we don't write any more, return null.
+        if (page == null) {
+            ioManager.close(fileHandle);
+            return null;
+        }
+
+        page.getBuffer().flip();
+
+        long delta = ioManager.syncWrite(fileHandle, resultState.getPersistentSize(), page.getBuffer());
+        resultState.incrementPersistentSize(delta);
+        return page;
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
new file mode 100644
index 0000000..3db3fd9
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.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.control.nc.dataset;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
+
+public class ResultState implements IStateObject {
+    private final ResultSetPartitionId resultSetPartitionId;
+
+    private final int frameSize;
+
+    private final IIOManager ioManager;
+
+    private final AtomicBoolean eos;
+
+    private final AtomicBoolean readEOS;
+
+    private final List<Page> localPageList;
+
+    private FileReference fileRef;
+
+    private long size;
+
+    private long persistentSize;
+
+    ResultState(ResultSetPartitionId resultSetPartitionId, IIOManager ioManager, int frameSize) {
+        this.resultSetPartitionId = resultSetPartitionId;
+        this.ioManager = ioManager;
+        this.frameSize = frameSize;
+        eos = new AtomicBoolean(false);
+        readEOS = new AtomicBoolean(false);
+        localPageList = new ArrayList<Page>();
+    }
+
+    public synchronized void init(FileReference fileRef) {
+        this.fileRef = fileRef;
+
+        size = 0;
+        persistentSize = 0;
+        notifyAll();
+    }
+
+    public ResultSetPartitionId getResultSetPartitionId() {
+        return resultSetPartitionId;
+    }
+
+    public int getFrameSize() {
+        return frameSize;
+    }
+
+    public IIOManager getIOManager() {
+        return ioManager;
+    }
+
+    public synchronized void incrementSize(long delta) {
+        size += delta;
+    }
+
+    public synchronized long getSize() {
+        return size;
+    }
+
+    public synchronized void incrementPersistentSize(long delta) {
+        persistentSize += delta;
+    }
+
+    public synchronized long getPersistentSize() {
+        return persistentSize;
+    }
+
+    public void setEOS(boolean eos) {
+        this.eos.set(eos);
+    }
+
+    public boolean getEOS() {
+        return eos.get();
+    }
+
+    public boolean getReadEOS() {
+        return readEOS.get();
+    }
+
+    public synchronized void addPage(Page page) {
+        localPageList.add(page);
+    }
+
+    public synchronized Page removePage(int index) {
+        Page page = null;
+        if (!localPageList.isEmpty()) {
+            page = localPageList.remove(index);
+        }
+        return page;
+    }
+
+    public synchronized Page getPage(int index) {
+        Page page = null;
+        if (!localPageList.isEmpty()) {
+            page = localPageList.get(index);
+        }
+        return page;
+    }
+
+    public synchronized Page getLastPage() {
+        Page page = null;
+        if (!localPageList.isEmpty()) {
+            page = localPageList.get(localPageList.size() - 1);
+        }
+        return page;
+    }
+
+    public synchronized Page getFirstPage() {
+        Page page = null;
+        if (!localPageList.isEmpty()) {
+            page = localPageList.get(0);
+        }
+        return page;
+    }
+
+    public synchronized FileReference getValidFileReference() throws InterruptedException {
+        while (fileRef == null)
+            wait();
+        return fileRef;
+    }
+
+    @Override
+    public JobId getJobId() {
+        return resultSetPartitionId.getJobId();
+    }
+
+    @Override
+    public Object getId() {
+        return resultSetPartitionId;
+    }
+
+    @Override
+    public long getMemoryOccupancy() {
+        throw new UnsupportedOperationException();
+    }
+
+    @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-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
new file mode 100644
index 0000000..5b8b333
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
@@ -0,0 +1,127 @@
+/*
+ * 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.control.nc.net;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.IChannelOpenListener;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
+
+public class DatasetNetworkManager implements IChannelConnectionFactory {
+    private static final Logger LOGGER = Logger.getLogger(DatasetNetworkManager.class.getName());
+
+    private static final int MAX_CONNECTION_ATTEMPTS = 5;
+
+    static final int INITIAL_MESSAGE_SIZE = 20;
+
+    private final IDatasetPartitionManager partitionManager;
+
+    private final MuxDemux md;
+
+    private NetworkAddress networkAddress;
+
+    public DatasetNetworkManager(InetAddress inetAddress, IDatasetPartitionManager partitionManager, int nThreads)
+            throws IOException {
+        this.partitionManager = partitionManager;
+        md = new MuxDemux(new InetSocketAddress(inetAddress, 0), new ChannelOpenListener(), nThreads,
+                MAX_CONNECTION_ATTEMPTS);
+    }
+
+    public void start() throws IOException {
+        md.start();
+        InetSocketAddress sockAddr = md.getLocalAddress();
+        networkAddress = new NetworkAddress(sockAddr.getAddress().getAddress(), sockAddr.getPort());
+    }
+
+    public NetworkAddress getNetworkAddress() {
+        return networkAddress;
+    }
+
+    public void stop() {
+
+    }
+
+    public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException {
+        MultiplexedConnection mConn = md.connect((InetSocketAddress) remoteAddress);
+        return mConn.openChannel();
+    }
+
+    private class ChannelOpenListener implements IChannelOpenListener {
+        @Override
+        public void channelOpened(ChannelControlBlock channel) {
+            channel.getReadInterface().setFullBufferAcceptor(new InitialBufferAcceptor(channel));
+            channel.getReadInterface().getEmptyBufferAcceptor().accept(ByteBuffer.allocate(INITIAL_MESSAGE_SIZE));
+        }
+    }
+
+    private class InitialBufferAcceptor implements ICloseableBufferAcceptor {
+        private final ChannelControlBlock ccb;
+
+        private NetworkOutputChannel noc;
+
+        public InitialBufferAcceptor(ChannelControlBlock ccb) {
+            this.ccb = ccb;
+        }
+
+        @Override
+        public void accept(ByteBuffer buffer) {
+            JobId jobId = new JobId(buffer.getLong());
+            int partition = buffer.getInt();
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Received initial dataset partition read request for JobId: " + jobId + " partition: "
+                        + partition + " on channel: " + ccb);
+            }
+            noc = new NetworkOutputChannel(ccb, 1);
+            try {
+                partitionManager.initializeDatasetPartitionReader(jobId, partition, noc);
+            } catch (HyracksException e) {
+                noc.abort();
+            }
+        }
+
+        @Override
+        public void close() {
+
+        }
+
+        @Override
+        public void error(int ecode) {
+            if (noc != null) {
+                noc.abort();
+            }
+        }
+    }
+
+    public MuxDemuxPerformanceCounters getPerformanceCounters() {
+        return md.getPerformanceCounters();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
index b805595..c8e4e94 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
@@ -27,6 +27,8 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
 import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
 import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
 import edu.uci.ics.hyracks.net.exceptions.NetException;
@@ -36,7 +38,7 @@
 import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
 import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
 
-public class NetworkManager {
+public class NetworkManager implements IChannelConnectionFactory {
     private static final Logger LOGGER = Logger.getLogger(NetworkManager.class.getName());
 
     private static final int MAX_CONNECTION_ATTEMPTS = 5;
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
index 16e31f7..ba6e6c3 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
@@ -21,7 +21,7 @@
 import edu.uci.ics.hyracks.api.channels.IInputChannel;
 import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.partitions.IPartition;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
@@ -82,7 +82,7 @@
     }
 
     @Override
-    public void open(IHyracksTaskContext ctx) throws HyracksDataException {
+    public void open(IHyracksCommonContext ctx) throws HyracksDataException {
         for (int i = 0; i < nBuffers; ++i) {
             emptyQueue.add(ctx.allocateFrame());
         }
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
index 45c091a..ea88a75 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
@@ -28,12 +28,12 @@
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.partitions.IPartition;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
 import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
 import edu.uci.ics.hyracks.control.common.job.PartitionState;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
-import edu.uci.ics.hyracks.control.nc.net.NetworkOutputChannel;
 import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
 
 public class PartitionManager {
@@ -98,7 +98,7 @@
         List<IPartition> pList = partitionMap.get(partitionId);
         if (pList != null && !pList.isEmpty()) {
             IPartition partition = pList.get(0);
-            writer.setTaskContext(partition.getTaskContext());
+            writer.setFrameSize(partition.getTaskContext().getFrameSize());
             partition.writeTo(writer);
             if (!partition.isReusable()) {
                 partitionMap.remove(partitionId);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
index bb9669d..7ed9d11 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
@@ -22,10 +22,10 @@
 import edu.uci.ics.hyracks.api.comm.PartitionChannel;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.NetworkInputChannel;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 import edu.uci.ics.hyracks.control.nc.Joblet;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
-import edu.uci.ics.hyracks.control.nc.net.NetworkInputChannel;
 
 public class ReportPartitionAvailabilityWork extends AbstractWork {
     private final NodeControllerService ncs;
diff --git a/hyracks/hyracks-data/hyracks-data-std/pom.xml b/hyracks/hyracks-data/hyracks-data-std/pom.xml
index 3f051f9..8f5f04e 100644
--- a/hyracks/hyracks-data/hyracks-data-std/pom.xml
+++ b/hyracks/hyracks-data/hyracks-data-std/pom.xml
@@ -16,8 +16,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/RawUTF8StringPointable.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/RawUTF8StringPointable.java
new file mode 100644
index 0000000..c90ce5a
--- /dev/null
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/RawUTF8StringPointable.java
@@ -0,0 +1,105 @@
+/*
+ * 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.data.std.primitive;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+/**
+ * This class provides the raw bytes-based comparison and hash function for UTF8 strings.
+ * Note that the comparison may not deliver the correct ordering for certain languages that include 2 or 3 bytes characters.
+ * But it works for single-byte character languages.
+ */
+public final class RawUTF8StringPointable extends AbstractPointable implements IHashable, IComparable {
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return false;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 0;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new RawUTF8StringPointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    @Override
+    public int compareTo(IPointable pointer) {
+        return compareTo(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+    }
+
+    @Override
+    public int compareTo(byte[] bytes, int start, int length) {
+        int utflen1 = UTF8StringPointable.getUTFLength(this.bytes, this.start);
+        int utflen2 = UTF8StringPointable.getUTFLength(bytes, start);
+
+        int c1 = 0;
+        int c2 = 0;
+
+        int s1Start = this.start + 2;
+        int s2Start = start + 2;
+
+        while (c1 < utflen1 && c2 < utflen2) {
+            char ch1 = (char) this.bytes[s1Start + c1];
+            char ch2 = (char) bytes[s2Start + c2];
+
+            if (ch1 != ch2) {
+                return ch1 - ch2;
+            }
+            c1++;
+            c2++;
+        }
+        return utflen1 - utflen2;
+    }
+
+    @Override
+    public int hash() {
+        int h = 0;
+        int utflen = UTF8StringPointable.getUTFLength(bytes, start);
+        int sStart = start + 2;
+        int c = 0;
+
+        while (c < utflen) {
+            char ch = (char) bytes[sStart + c];
+            h = 31 * h + ch;
+            c++;
+        }
+        return h;
+    }
+
+    public void toString(StringBuilder buffer) {
+        UTF8StringPointable.toString(buffer, bytes, start);
+    }
+}
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java
index f6d6093..866ebb0 100644
--- a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java
@@ -216,4 +216,4 @@
     public void toString(StringBuilder buffer) {
         toString(buffer, bytes, start);
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ArrayBackedValueStorage.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ArrayBackedValueStorage.java
index e8dc9b4..7d7d2c1 100644
--- a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ArrayBackedValueStorage.java
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ArrayBackedValueStorage.java
@@ -1,34 +1,28 @@
 package edu.uci.ics.hyracks.data.std.util;
 
 import java.io.DataOutput;
-import java.io.DataOutputStream;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.data.std.api.IMutableValueStorage;
 import edu.uci.ics.hyracks.data.std.api.IValueReference;
 
 public class ArrayBackedValueStorage implements IMutableValueStorage {
-    private final ByteArrayAccessibleOutputStream baaos;
-    private final DataOutputStream dos;
-
-    public ArrayBackedValueStorage() {
-        baaos = new ByteArrayAccessibleOutputStream();
-        dos = new DataOutputStream(baaos);
-    }
+   
+    private final GrowableArray data = new GrowableArray();
 
     @Override
     public void reset() {
-        baaos.reset();
+        data.reset();
     }
 
     @Override
     public DataOutput getDataOutput() {
-        return dos;
+        return data.getDataOutput();
     }
 
     @Override
     public byte[] getByteArray() {
-        return baaos.getByteArray();
+        return data.getByteArray();
     }
 
     @Override
@@ -38,12 +32,12 @@
 
     @Override
     public int getLength() {
-        return baaos.size();
+        return data.getLength();
     }
 
     public void append(IValueReference value) {
         try {
-            dos.write(value.getByteArray(), value.getStartOffset(), value.getLength());
+            data.append(value);
         } catch (IOException e) {
             e.printStackTrace();
         }
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
index 8508287..c0ba163 100644
--- a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
@@ -21,6 +21,14 @@
 public class ByteArrayAccessibleOutputStream extends ByteArrayOutputStream {
     private static final Logger LOGGER = Logger.getLogger(ByteArrayAccessibleOutputStream.class.getName());
 
+    public ByteArrayAccessibleOutputStream() {
+        super();
+    }
+
+    public ByteArrayAccessibleOutputStream(int size) {
+        super(size);
+    }
+
     public byte[] getByteArray() {
         return buf;
     }
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/GrowableArray.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/GrowableArray.java
new file mode 100644
index 0000000..c174d4e
--- /dev/null
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/GrowableArray.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2012 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.data.std.util;
+
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.api.IValueReference;
+
+public class GrowableArray implements IDataOutputProvider {
+    private final ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
+    private final DataOutputStream dos = new DataOutputStream(baaos);
+
+    @Override
+    public DataOutput getDataOutput() {
+        return dos;
+    }
+
+    public void reset() {
+        baaos.reset();
+    }
+
+    public byte[] getByteArray() {
+        return baaos.getByteArray();
+    }
+
+    public int getLength() {
+        return baaos.size();
+    }
+
+    public void append(IValueReference value) throws IOException {
+        dos.write(value.getByteArray(), value.getStartOffset(), value.getLength());
+    }
+}
diff --git a/hyracks/hyracks-dataflow-common/pom.xml b/hyracks/hyracks-dataflow-common/pom.xml
index a0ffb66..1a2950b 100644
--- a/hyracks/hyracks-dataflow-common/pom.xml
+++ b/hyracks/hyracks-dataflow-common/pom.xml
@@ -15,8 +15,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
index 989bc6b..8c865c4 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
@@ -15,14 +15,13 @@
 package edu.uci.ics.hyracks.dataflow.common.comm.io;
 
 import java.io.DataOutput;
-import java.io.DataOutputStream;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 
 /**
  * Array backed tuple builder.
@@ -30,25 +29,21 @@
  * @author vinayakb
  */
 public class ArrayTupleBuilder implements IDataOutputProvider {
-    private final ByteArrayAccessibleOutputStream baaos;
-    private final DataOutputStream dos;
+    private final GrowableArray fieldData = new GrowableArray();
     private final int[] fEndOffsets;
     private int nextField;
 
     public ArrayTupleBuilder(int nFields) {
-        baaos = new ByteArrayAccessibleOutputStream();
-        dos = new DataOutputStream(baaos);
         fEndOffsets = new int[nFields];
     }
 
     /**
      * Resets the builder.
-     * 
      * reset() must be called before attempting to create a new tuple.
      */
     public void reset() {
         nextField = 0;
-        baaos.reset();
+        fieldData.reset();
     }
 
     /**
@@ -66,7 +61,7 @@
      * @return Data byte array.
      */
     public byte[] getByteArray() {
-        return baaos.getByteArray();
+        return fieldData.getByteArray();
     }
 
     /**
@@ -75,7 +70,7 @@
      * @return data area size.
      */
     public int getSize() {
-        return baaos.size();
+        return fieldData.getLength();
     }
 
     /**
@@ -96,14 +91,15 @@
         int fStartOffset = accessor.getFieldStartOffset(tIndex, fIndex);
         int fLen = accessor.getFieldEndOffset(tIndex, fIndex) - fStartOffset;
         try {
-            dos.write(accessor.getBuffer().array(), startOffset + accessor.getFieldSlotsLength() + fStartOffset, fLen);
+            fieldData.getDataOutput().write(accessor.getBuffer().array(),
+                    startOffset + accessor.getFieldSlotsLength() + fStartOffset, fLen);
             if (FrameConstants.DEBUG_FRAME_IO) {
-                dos.writeInt(FrameConstants.FRAME_FIELD_MAGIC);
+                fieldData.getDataOutput().writeInt(FrameConstants.FRAME_FIELD_MAGIC);
             }
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
-        fEndOffsets[nextField++] = baaos.size();
+        fEndOffsets[nextField++] = fieldData.getLength();
     }
 
     /**
@@ -117,8 +113,8 @@
      * @throws HyracksDataException
      */
     public <T> void addField(ISerializerDeserializer<T> serDeser, T instance) throws HyracksDataException {
-        serDeser.serialize(instance, dos);
-        fEndOffsets[nextField++] = baaos.size();
+        serDeser.serialize(instance, fieldData.getDataOutput());
+        fEndOffsets[nextField++] = fieldData.getLength();
     }
 
     /**
@@ -134,11 +130,11 @@
      */
     public void addField(byte[] bytes, int start, int length) throws HyracksDataException {
         try {
-            dos.write(bytes, start, length);
+            fieldData.getDataOutput().write(bytes, start, length);
         } catch (IOException e) {
             throw new HyracksDataException(e);
         }
-        fEndOffsets[nextField++] = baaos.size();
+        fEndOffsets[nextField++] = fieldData.getLength();
     }
 
     /**
@@ -146,7 +142,14 @@
      */
     @Override
     public DataOutput getDataOutput() {
-        return dos;
+        return fieldData.getDataOutput();
+    }
+
+    /**
+     * Get the growable array storing the field data.
+     */
+    public GrowableArray getFieldData() {
+        return fieldData;
     }
 
     /**
@@ -156,6 +159,6 @@
      * data.
      */
     public void addFieldEndOffset() {
-        fEndOffsets[nextField++] = baaos.size();
+        fEndOffsets[nextField++] = fieldData.getLength();
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
new file mode 100644
index 0000000..07f6ba2
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
@@ -0,0 +1,54 @@
+/*
+ * 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.common.comm.io;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+
+public class FrameOutputStream extends ByteArrayAccessibleOutputStream {
+    private static final Logger LOGGER = Logger.getLogger(FrameOutputStream.class.getName());
+
+    private final FrameTupleAppender frameTupleAppender;
+
+    public FrameOutputStream(int frameSize) {
+        super(frameSize);
+        this.frameTupleAppender = new FrameTupleAppender(frameSize);
+    }
+
+    public void reset(ByteBuffer buffer, boolean clear) {
+        frameTupleAppender.reset(buffer, clear);
+    }
+
+    public int getTupleCount() {
+        int tupleCount = frameTupleAppender.getTupleCount();
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("appendTuple(): tuple count: " + tupleCount);
+        }
+        return tupleCount;
+    }
+
+    public boolean appendTuple() {
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("appendTuple(): tuple size: " + count);
+        }
+        boolean appended = frameTupleAppender.append(buf, 0, count);
+        count = 0;
+        return appended;
+    }
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
new file mode 100644
index 0000000..915a436
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.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.common.comm.io;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+public class ResultFrameTupleAccessor implements IFrameTupleAccessor {
+
+    private final int frameSize;
+    private ByteBuffer buffer;
+
+    public ResultFrameTupleAccessor(int frameSize) {
+        this.frameSize = frameSize;
+    }
+
+    @Override
+    public void reset(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    @Override
+    public int getTupleCount() {
+        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
+    }
+
+    @Override
+    public int getTupleStartOffset(int tupleIndex) {
+        return tupleIndex == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * tupleIndex);
+    }
+
+    @Override
+    public int getTupleEndOffset(int tupleIndex) {
+        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleIndex + 1));
+    }
+
+    @Override
+    public int getFieldStartOffset(int tupleIndex, int fIdx) {
+        return fIdx == 0 ? 0 : buffer.getInt(getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
+    }
+
+    @Override
+    public int getFieldEndOffset(int tupleIndex, int fIdx) {
+        return buffer.getInt(getTupleStartOffset(tupleIndex) + fIdx * 4);
+    }
+
+    @Override
+    public int getFieldLength(int tupleIndex, int fIdx) {
+        return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
+    }
+
+    @Override
+    public int getFieldSlotsLength() {
+        return getFieldCount() * 4;
+    }
+
+    public void prettyPrint() {
+        ByteBufferInputStream bbis = new ByteBufferInputStream();
+        DataInputStream dis = new DataInputStream(bbis);
+        int tc = getTupleCount();
+        System.err.println("TC: " + tc);
+        for (int i = 0; i < tc; ++i) {
+            System.err.print(i + ":(" + getTupleStartOffset(i) + ", " + getTupleEndOffset(i) + ")[");
+
+            bbis.setByteBuffer(buffer, getTupleStartOffset(i));
+            System.err.print(dis);
+
+            System.err.println("]");
+        }
+    }
+
+    @Override
+    public int getFieldCount() {
+        return 1;
+    }
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/DoubleNormalizedKeyComputerFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/DoubleNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..e95e9c2
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/DoubleNormalizedKeyComputerFactory.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.hyracks.dataflow.common.data.normalizers;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class DoubleNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public INormalizedKeyComputer createNormalizedKeyComputer() {
+        return new INormalizedKeyComputer() {
+
+            @Override
+            public int normalize(byte[] bytes, int start, int length) {
+                int prefix = IntegerSerializerDeserializer.getInt(bytes, start);
+                if (prefix >= 0) {
+                    return prefix ^ Integer.MIN_VALUE;
+                } else {
+                    return (int) ((long) 0xffffffff - (long) prefix);
+                }
+            }
+
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/FloatNormalizedKeyComputerFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/FloatNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..d58afc1
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/FloatNormalizedKeyComputerFactory.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.hyracks.dataflow.common.data.normalizers;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class FloatNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public INormalizedKeyComputer createNormalizedKeyComputer() {
+        return new INormalizedKeyComputer() {
+
+            @Override
+            public int normalize(byte[] bytes, int start, int length) {
+                int prefix = IntegerSerializerDeserializer.getInt(bytes, start);
+                if (prefix >= 0) {
+                    return prefix ^ Integer.MIN_VALUE;
+                } else {
+                    return (int) ((long) 0xffffffff - (long) prefix);
+                }
+            }
+
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/Integer64NormalizedKeyComputerFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/Integer64NormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..4589909
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/Integer64NormalizedKeyComputerFactory.java
@@ -0,0 +1,55 @@
+package edu.uci.ics.hyracks.dataflow.common.data.normalizers;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.Integer64SerializerDeserializer;
+
+public class Integer64NormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+    private static final long serialVersionUID = 8735044913496854551L;
+
+    @Override
+    public INormalizedKeyComputer createNormalizedKeyComputer() {
+        return new INormalizedKeyComputer() {
+            private static final int POSTIVE_LONG_MASK = (3 << 30);
+            private static final int NON_NEGATIVE_INT_MASK = (2 << 30);
+            private static final int NEGATIVE_LONG_MASK = (0 << 30);
+
+            @Override
+            public int normalize(byte[] bytes, int start, int length) {
+                long value = Integer64SerializerDeserializer.getLong(bytes, start);
+                int highValue = (int) (value >> 32);
+                if (highValue > 0) {
+                    /**
+                     * larger than Integer.MAX
+                     */
+                    int highNmk = getKey(highValue);
+                    highNmk >>= 2;
+                    highNmk |= POSTIVE_LONG_MASK;
+                    return highNmk;
+                } else if (highValue == 0) {
+                    /**
+                     * smaller than Integer.MAX but >=0
+                     */
+                    int lowNmk = (int) value;
+                    lowNmk >>= 2;
+                    lowNmk |= NON_NEGATIVE_INT_MASK;
+                    return lowNmk;
+                } else {
+                    /**
+                     * less than 0: have not optimized for that
+                     */
+                    int highNmk = getKey(highValue);
+                    highNmk >>= 2;
+                    highNmk |= NEGATIVE_LONG_MASK;
+                    return highNmk;
+                }
+            }
+
+            private int getKey(int value) {
+                return value ^ Integer.MIN_VALUE;
+            }
+
+        };
+    }
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactory.java
index 2f7a778..6a01842 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactory.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactory.java
@@ -27,8 +27,7 @@
             @Override
             public int normalize(byte[] bytes, int start, int length) {
                 int value = IntegerSerializerDeserializer.getInt(bytes, start);
-                long unsignedValue = (long) value;
-                return (int) ((unsignedValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
+                return value ^Integer.MIN_VALUE;
             }
         };
     }
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/parsers/LongParserFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/parsers/LongParserFactory.java
index 6a9dab7..d9191c7 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/parsers/LongParserFactory.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/parsers/LongParserFactory.java
@@ -32,7 +32,7 @@
         return new IValueParser() {
             @Override
             public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
-                long n = 0;
+                int n = 0;
                 int sign = 1;
                 int i = 0;
                 boolean pre = true;
@@ -102,7 +102,7 @@
                             throw new HyracksDataException("Encountered " + ch);
                     }
                 }
-
+                
                 try {
                     out.writeLong(n * sign);
                 } catch (IOException e) {
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
index 51645c4..ec3c2be 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
@@ -52,10 +52,10 @@
                     h += fh;
                 }
                 if (h < 0) {
-                    h = -h;
+                    h = -(h+1);
                 }
                 return h % nParts;
             }
         };
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
index a29209c..8a30a71 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
@@ -18,17 +18,19 @@
 import java.io.IOException;
 
 public class StringUtils {
-    public static void writeCharAsModifiedUTF8(char c, DataOutput dos) throws IOException {
-
+    public static int writeCharAsModifiedUTF8(char c, DataOutput dos) throws IOException {
         if (c >= 0x0000 && c <= 0x007F) {
             dos.writeByte(c);
+            return 1;
         } else if (c <= 0x07FF) {
             dos.writeByte((byte) (0xC0 | ((c >> 6) & 0x3F)));
             dos.writeByte((byte) (0x80 | (c & 0x3F)));
+            return 2;
         } else {
             dos.writeByte((byte) (0xE0 | ((c >> 12) & 0x0F)));
             dos.writeByte((byte) (0x80 | ((c >> 6) & 0x3F)));
             dos.writeByte((byte) (0x80 | (c & 0x3F)));
+            return 3;
         }
     }
 
diff --git a/hyracks/hyracks-dataflow-hadoop/pom.xml b/hyracks/hyracks-dataflow-hadoop/pom.xml
index f9d5153..f5135f8 100644
--- a/hyracks/hyracks-dataflow-hadoop/pom.xml
+++ b/hyracks/hyracks-dataflow-hadoop/pom.xml
@@ -1,8 +1,6 @@
 <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-hadoop</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hyracks-dataflow-hadoop</name>
 
   <parent>
@@ -18,8 +16,8 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-dataflow-std/pom.xml b/hyracks/hyracks-dataflow-std/pom.xml
index bf27d20..2cf0fdc 100644
--- a/hyracks/hyracks-dataflow-std/pom.xml
+++ b/hyracks/hyracks-dataflow-std/pom.xml
@@ -1,8 +1,6 @@
 <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.3-SNAPSHOT</version>
   <name>hyracks-dataflow-std</name>
 
   <parent>
@@ -18,8 +16,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index f2b56fa..f86d9fb 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -256,8 +256,6 @@
 
                 outputAppender.reset(outputFrame, true);
 
-                writer.open();
-
                 if (tPointers == null) {
                     // Not sorted
                     for (int i = 0; i < tableSize; ++i) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
index 3e5e30f..d86f1d5 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -17,7 +17,6 @@
 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;
@@ -47,11 +46,18 @@
     private final ISerializableTable table;
 	private final int tableSize;
     private final TuplePointer storedTuplePointer;
+    private final boolean reverseOutputOrder;	//Should we reverse the order of tuples, we are writing in output
     
     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(ctx, tableSize, accessor0, tpc0, accessor1, tpc1, comparator, isLeftOuter, nullWriters1, table, false);
+    }
+    
+    public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
+            ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
+            FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1, ISerializableTable table, boolean reverse) throws HyracksDataException {
     	this.tableSize = tableSize;
        	this.table = table;
        	storedTuplePointer = new TuplePointer();
@@ -76,6 +82,7 @@
         } else {
             nullTupleBuild = null;
         }
+    	reverseOutputOrder = reverse;
     }
 
     public void build(ByteBuffer buffer) throws HyracksDataException {
@@ -108,18 +115,13 @@
                 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();
-                        }
-                    }
+                    appendToResult(i, tIndex, writer);
                 }
             } while (true);
 
             if (!matchFound && isLeftOuter) {
-                if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
+                
+            	if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
                         nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
                     flushFrame(outBuffer, writer);
                     appender.reset(outBuffer, true);
@@ -128,6 +130,7 @@
                         throw new IllegalStateException();
                     }
                 }
+                
             }
         }
     }
@@ -145,23 +148,25 @@
         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);
+    
+    private void appendToResult(int probeSidetIx, int buildSidetIx, IFrameWriter writer) throws HyracksDataException{
+    	if(!reverseOutputOrder){
+    		if (!appender.appendConcat(accessorProbe, probeSidetIx, accessorBuild, buildSidetIx)) {
+                flushFrame(outBuffer, writer);
+                appender.reset(outBuffer, true);
+                if (!appender.appendConcat(accessorProbe, probeSidetIx, accessorBuild, buildSidetIx)) {
+                    throw new IllegalStateException();
+                }
             }
-            pointers[size++] = pointer;
-        }
+    	}
+    	else{
+    		if (!appender.appendConcat(accessorBuild, buildSidetIx, accessorProbe, probeSidetIx)) {
+                flushFrame(outBuffer, writer);
+                appender.reset(outBuffer, true);
+                if (!appender.appendConcat(accessorBuild, buildSidetIx, accessorProbe, probeSidetIx)) {
+                    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/join/NestedLoopJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
index 7e84229..6870e71 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -14,15 +14,18 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.join;
 
+import java.io.DataOutput;
 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.INullWriter;
 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.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;
@@ -42,9 +45,12 @@
     private RunFileReader runFileReader;
     private int currentMemSize = 0;
     private final RunFileWriter runFileWriter;
+    private final boolean isLeftOuter;
+    private final ArrayTupleBuilder nullTupleBuilder;
 
     public NestedLoopJoin(IHyracksTaskContext ctx, FrameTupleAccessor accessor0, FrameTupleAccessor accessor1,
-            ITuplePairComparator comparators, int memSize) throws HyracksDataException {
+            ITuplePairComparator comparators, int memSize, boolean isLeftOuter, INullWriter[] nullWriters1)
+            throws HyracksDataException {
         this.accessorInner = accessor1;
         this.accessorOuter = accessor0;
         this.appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -56,6 +62,19 @@
         this.memSize = memSize;
         this.ctx = ctx;
 
+        this.isLeftOuter = isLeftOuter;
+        if (isLeftOuter) {
+            int innerFieldCount = accessorInner.getFieldCount();
+            nullTupleBuilder = new ArrayTupleBuilder(innerFieldCount);
+            DataOutput out = nullTupleBuilder.getDataOutput();
+            for (int i = 0; i < innerFieldCount; i++) {
+                nullWriters1[i].writeNull(out);
+                nullTupleBuilder.addFieldEndOffset();
+            }
+        } else {
+            nullTupleBuilder = null;
+        }
+
         FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
                 this.getClass().getSimpleName() + this.toString());
         runFileWriter = new RunFileWriter(file, ctx.getIOManager());
@@ -108,9 +127,11 @@
         int tupleCount1 = accessorInner.getTupleCount();
 
         for (int i = 0; i < tupleCount0; ++i) {
+            boolean matchFound = false;
             for (int j = 0; j < tupleCount1; ++j) {
                 int c = compare(accessorOuter, i, accessorInner, j);
                 if (c == 0) {
+                    matchFound = true;
                     if (!appender.appendConcat(accessorOuter, i, accessorInner, j)) {
                         flushFrame(outBuffer, writer);
                         appender.reset(outBuffer, true);
@@ -120,6 +141,18 @@
                     }
                 }
             }
+
+            if (!matchFound && isLeftOuter) {
+                if (!appender.appendConcat(accessorOuter, i, nullTupleBuilder.getFieldEndOffsets(),
+                        nullTupleBuilder.getByteArray(), 0, nullTupleBuilder.getSize())) {
+                    flushFrame(outBuffer, writer);
+                    appender.reset(outBuffer, true);
+                    if (!appender.appendConcat(accessorOuter, i, nullTupleBuilder.getFieldEndOffsets(),
+                            nullTupleBuilder.getByteArray(), 0, nullTupleBuilder.getSize())) {
+                        throw new IllegalStateException();
+                    }
+                }
+            }
         }
     }
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
index a699703..0be01c1 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
@@ -25,6 +25,8 @@
 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.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;
@@ -47,13 +49,18 @@
     private static final long serialVersionUID = 1L;
     private final ITuplePairComparatorFactory comparatorFactory;
     private final int memSize;
+    private final boolean isLeftOuter;
+    private final INullWriterFactory[] nullWriterFactories1;
 
     public NestedLoopJoinOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize) {
+            ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize,
+            boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
         super(spec, 2, 1);
         this.comparatorFactory = comparatorFactory;
         this.recordDescriptors[0] = recordDescriptor;
         this.memSize = memSize;
+        this.isLeftOuter = isLeftOuter;
+        this.nullWriterFactories1 = nullWriterFactories1;
     }
 
     @Override
@@ -111,6 +118,13 @@
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
             final ITuplePairComparator comparator = comparatorFactory.createTuplePairComparator(ctx);
 
+            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 JoinCacheTaskState state;
 
@@ -118,8 +132,11 @@
                 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);
+                            new FrameTupleAccessor(ctx.getFrameSize(), rd1), comparator, memSize, isLeftOuter,
+                            nullWriters1);
+
                 }
 
                 @Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index 2905574..6e2b16a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -81,6 +81,8 @@
 
     private int[] buildPSizeInFrames; //Used for partition tuning
     private int freeFramesCounter; //Used for partition tuning
+    
+    private boolean isTableEmpty;	//Added for handling the case, where build side is empty (tableSize is 0)
 
     public OptimizedHybridHashJoin(IHyracksTaskContext ctx, int memForJoin, int numOfPartitions, String rel0Name,
             String rel1Name, int[] keys0, int[] keys1, IBinaryComparator[] comparators, RecordDescriptor buildRd,
@@ -89,10 +91,10 @@
         this.memForJoin = memForJoin;
         this.buildRd = buildRd;
         this.probeRd = probeRd;
-        this.buildHpc = probeHpc;
-        this.probeHpc = buildHpc;
-        this.buildKeys = keys0;
-        this.probeKeys = keys1;
+        this.buildHpc = buildHpc; 	
+        this.probeHpc = probeHpc; 	
+        this.buildKeys = keys1; 	
+        this.probeKeys = keys0;		
         this.comparators = comparators;
         this.rel0Name = rel0Name;
         this.rel1Name = rel1Name;
@@ -117,10 +119,10 @@
         this.memForJoin = memForJoin;
         this.buildRd = buildRd;
         this.probeRd = probeRd;
-        this.buildHpc = probeHpc;
-        this.probeHpc = buildHpc;
-        this.buildKeys = keys0;
-        this.probeKeys = keys1;
+        this.buildHpc = buildHpc; 	
+        this.probeHpc = probeHpc; 	
+        this.buildKeys = keys1; 	
+        this.probeKeys = keys0;		
         this.comparators = comparators;
         this.rel0Name = rel0Name;
         this.rel1Name = rel1Name;
@@ -171,6 +173,12 @@
     public void build(ByteBuffer buffer) throws HyracksDataException {
         accessorBuild.reset(buffer);
         int tupleCount = accessorBuild.getTupleCount();
+   
+        boolean print = false;
+    	if(print){
+    		accessorBuild.prettyPrint();
+    	}
+        
         for (int i = 0; i < tupleCount; ++i) {
             int pid = buildHpc.partition(accessorBuild, i, numOfPartitions);
             processTuple(i, pid);
@@ -338,6 +346,7 @@
 
         createInMemoryJoiner(inMemTupCount);
         cacheInMemJoin();
+        this.isTableEmpty = (inMemTupCount == 0);
     }
 
     private void partitionTune() throws HyracksDataException {
@@ -457,10 +466,14 @@
     }
 
     public void probe(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
-
         accessorProbe.reset(buffer);
         int tupleCount = accessorProbe.getTupleCount();
 
+        boolean print = false;
+    	if(print){
+    		accessorProbe.prettyPrint();
+    	}
+        
         if (numOfSpilledParts == 0) {
             inMemJoiner.join(buffer, writer);
             return;
@@ -604,4 +617,8 @@
                 + freeFramesCounter;
         return s;
     }
+    
+    public boolean isTableEmpty(){
+    	return this.isTableEmpty;
+    }
 }
\ 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
index 3a7ee2c..cf39416 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -31,6 +31,7 @@
 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.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
@@ -135,6 +136,7 @@
         recordDescriptors[0] = recordDescriptor;
         this.isLeftOuter = isLeftOuter;
         this.nullWriterFactories1 = nullWriterFactories1;
+        
 
     }
 
@@ -167,10 +169,10 @@
         ProbeAndJoinActivityNode phase2 = new ProbeAndJoinActivityNode(probeAid, buildAid);
 
         builder.addActivity(this, phase1);
-        builder.addSourceEdge(0, phase1, 0);
+        builder.addSourceEdge(1, phase1, 0);
 
         builder.addActivity(this, phase2);
-        builder.addSourceEdge(1, phase2, 0);
+        builder.addSourceEdge(0, phase2, 0);
 
         builder.addBlockingEdge(phase1, phase2);
 
@@ -253,14 +255,8 @@
             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));
@@ -278,9 +274,17 @@
                     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);
+                    if(!isLeftOuter){
+                    	state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
+                                PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
+                                buildHpc);
+                    }
+                    else{
+                    	state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
+                                PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
+                                buildHpc, isLeftOuter, nullWriterFactories1);
+                    }
+                    
                     state.hybridHJ.initBuild();
                 }
 
@@ -368,7 +372,9 @@
 
                 @Override
                 public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    state.hybridHJ.probe(buffer, writer);
+                	if(!state.hybridHJ.isTableEmpty()){
+                		state.hybridHJ.probe(buffer, writer);
+                	}
                 }
 
                 @Override
@@ -418,34 +424,40 @@
                     //Apply in-Mem HJ if possible
                     if ((buildPartSize < state.memForJoin) || (probePartSize < state.memForJoin)) {
                         int tabSize = -1;
-                        if (buildPartSize < probePartSize) {
+                        
+                        if (isLeftOuter || 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);
+                          //Build Side is smaller
+                            applyInMemHashJoin(buildKeys, probeKeys, tabSize, probeRd, buildRd, hpcRep0, hpcRep1,
+                                    buildSideReader, probeSideReader, false, pid);
 
-                        } else { //Role Reversal
+                        } 
+                        
+                        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);
+                            
+                            applyInMemHashJoin(probeKeys, buildKeys, tabSize, buildRd, probeRd, hpcRep1, hpcRep0,
+                                    probeSideReader, buildSideReader, true, pid);
                         }
                     }
                     //Apply (Recursive) HHJ
                     else {
                         OptimizedHybridHashJoin rHHj;
-                        if (buildPartSize < probePartSize) { //Build Side is smaller
+                        if (isLeftOuter || 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);
 
@@ -488,14 +500,14 @@
                                 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) {
+                                    if (isLeftOuter || buildSideInTups < probeSideInTups) {
                                         applyNestedLoopJoin(probeRd, buildRd, state.memForJoin, rbrfw, rprfw,
                                                 nljComparator0);
                                     } else {
@@ -507,6 +519,7 @@
                         } 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);
 
@@ -545,7 +558,7 @@
                                 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;
                                     }
@@ -569,14 +582,14 @@
 
                 private void applyInMemHashJoin(int[] bKeys, int[] pKeys, int tabSize, RecordDescriptor buildRDesc,
                         RecordDescriptor probeRDesc, ITuplePartitionComputer hpcRepLarger,
-                        ITuplePartitionComputer hpcRepSmaller, RunFileReader bReader, RunFileReader pReader)
+                        ITuplePartitionComputer hpcRepSmaller, RunFileReader bReader, RunFileReader pReader, boolean reverse, int pid)
                         throws HyracksDataException {
 
                     ISerializableTable table = new SerializableHashTable(tabSize, ctx);
                     InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tabSize, new FrameTupleAccessor(
                             ctx.getFrameSize(), probeRDesc), hpcRepLarger, new FrameTupleAccessor(ctx.getFrameSize(),
                             buildRDesc), hpcRepSmaller, new FrameTuplePairComparator(pKeys, bKeys, comparators),
-                            isLeftOuter, nullWriters1, table);
+                            isLeftOuter, nullWriters1, table, reverse);
 
                     bReader.open();
                     rPartbuff.clear();
@@ -604,7 +617,7 @@
                         throws HyracksDataException {
 
                     NestedLoopJoin nlj = new NestedLoopJoin(ctx, new FrameTupleAccessor(ctx.getFrameSize(), outerRd),
-                            new FrameTupleAccessor(ctx.getFrameSize(), innerRd), nljComparator, memorySize);
+                            new FrameTupleAccessor(ctx.getFrameSize(), innerRd), nljComparator, memorySize, false, null);
 
                     ByteBuffer cacheBuff = ctx.allocateFrame();
                     innerReader.open();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
new file mode 100644
index 0000000..edca60a
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.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.result;
+
+import java.io.IOException;
+import java.io.PrintStream;
+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.IResultSerializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+
+public class ResultWriterOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final ResultSetId rsId;
+
+    private final boolean ordered;
+
+    private final IResultSerializerFactory resultSerializerFactory;
+
+    public ResultWriterOperatorDescriptor(IOperatorDescriptorRegistry spec, ResultSetId rsId, boolean ordered,
+            IResultSerializerFactory resultSerializerFactory) throws IOException {
+        super(spec, 1, 0);
+        this.rsId = rsId;
+        this.ordered = ordered;
+        this.resultSerializerFactory = resultSerializerFactory;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+        final IDatasetPartitionManager dpm = ctx.getDatasetPartitionManager();
+
+        final ByteBuffer outputBuffer = ctx.allocateFrame();
+
+        final FrameOutputStream frameOutputStream = new FrameOutputStream(ctx.getFrameSize());
+        frameOutputStream.reset(outputBuffer, true);
+        PrintStream printStream = new PrintStream(frameOutputStream);
+
+        final RecordDescriptor outRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+        final IResultSerializer resultSerializer = resultSerializerFactory.createResultSerializer(outRecordDesc,
+                printStream);
+
+        final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), outRecordDesc);
+
+        return new AbstractUnaryInputSinkOperatorNodePushable() {
+            IFrameWriter datasetPartitionWriter;
+
+            @Override
+            public void open() throws HyracksDataException {
+                try {
+                    datasetPartitionWriter = dpm.createDatasetPartitionWriter(ctx, rsId, ordered, partition,
+                            nPartitions);
+                    datasetPartitionWriter.open();
+                    resultSerializer.init();
+                } catch (HyracksException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                frameTupleAccessor.reset(buffer);
+                for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+                    resultSerializer.appendTuple(frameTupleAccessor, tIndex);
+                    if (!frameOutputStream.appendTuple()) {
+                        datasetPartitionWriter.nextFrame(outputBuffer);
+                        frameOutputStream.reset(outputBuffer, true);
+
+                        /* TODO(madhusudancs): This works under the assumption that no single serialized record is
+                         * longer than the buffer size.
+                         */
+                        resultSerializer.appendTuple(frameTupleAccessor, tIndex);
+                        frameOutputStream.appendTuple();
+                    }
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                datasetPartitionWriter.fail();
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                if (frameOutputStream.getTupleCount() > 0) {
+                    datasetPartitionWriter.nextFrame(outputBuffer);
+                    frameOutputStream.reset(outputBuffer, true);
+                }
+                datasetPartitionWriter.close();
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dist/pom.xml b/hyracks/hyracks-dist/pom.xml
new file mode 100755
index 0000000..58a4b1c
--- /dev/null
+++ b/hyracks/hyracks-dist/pom.xml
@@ -0,0 +1,74 @@
+<?xml version="1.0"?>
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<artifactId>hyracks</artifactId>
+		<groupId>edu.uci.ics.hyracks</groupId>
+		<version>0.2.3-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>hyracks-dist</artifactId>
+	<name>hyracks-dist</name>
+
+	<properties>
+		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+	</properties>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-compiler-plugin</artifactId>
+				<version>2.0.2</version>
+				<configuration>
+					<source>1.7</source>
+					<target>1.7</target>
+				</configuration>
+			</plugin>
+			<plugin>
+				<artifactId>maven-resources-plugin</artifactId>
+				<version>2.5</version>
+				<executions>
+					<execution>
+						<id>copy-scripts</id>
+						<!-- here the phase you need -->
+						<phase>package</phase>
+						<goals>
+							<goal>copy-resources</goal>
+						</goals>
+						<configuration>
+							<outputDirectory>target/appassembler/</outputDirectory>
+							<resources>
+								<resource>
+									<directory>src/main/resources</directory>
+								</resource>
+							</resources>
+							<directoryMode>0755</directoryMode>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-antrun-plugin</artifactId>
+				<version>1.6</version>
+				<executions>
+					<execution>
+						<id>process-test-classes</id>
+						<phase>package</phase>
+						<configuration>
+							<target>
+								<chmod file="target/appassembler/bin/*)" perm="755" />
+							</target>
+						</configuration>
+						<goals>
+							<goal>run</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/getip.sh b/hyracks/hyracks-dist/src/main/resources/bin/getip.sh
new file mode 100755
index 0000000..a691c0f
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/getip.sh
@@ -0,0 +1,25 @@
+#get the OS
+OS_NAME=`uname -a|awk '{print $1}'`
+LINUX_OS='Linux'
+
+if [ $OS_NAME = $LINUX_OS ];
+then
+        #Get IP Address
+        IPADDR=`/sbin/ifconfig eth0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+    	if [ "$IPADDR" = "" ]
+        then
+		IPADDR=`/sbin/ifconfig em1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+        fi 
+	if [ "$IPADDR" = "" ]
+        then
+		IPADDR=`/sbin/ifconfig lo | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+        fi 
+else
+        IPADDR=`/sbin/ifconfig en1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+	if [ "$IPADDR" = "" ]
+        then
+                IPADDR=`/sbin/ifconfig lo0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+        fi
+
+fi
+echo $IPADDR
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/startAllNCs.sh b/hyracks/hyracks-dist/src/main/resources/bin/startAllNCs.sh
new file mode 100755
index 0000000..629bd90
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/startAllNCs.sh
@@ -0,0 +1,6 @@
+PREGELIX_PATH=`pwd`
+
+for i in `cat conf/slaves`
+do
+   ssh $i "cd ${PREGELIX_PATH}; bin/startnc.sh"
+done
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/startCluster.sh b/hyracks/hyracks-dist/src/main/resources/bin/startCluster.sh
new file mode 100755
index 0000000..a0c2063
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/startCluster.sh
@@ -0,0 +1,3 @@
+bin/startcc.sh
+sleep 5
+bin/startAllNCs.sh
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/startDebugNc.sh b/hyracks/hyracks-dist/src/main/resources/bin/startDebugNc.sh
new file mode 100755
index 0000000..fe6cf27
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/startDebugNc.sh
@@ -0,0 +1,50 @@
+hostname
+
+#Get the IP address of the cc
+CCHOST_NAME=`cat conf/master`
+CURRENT_PATH=`pwd`
+CCHOST=`ssh ${CCHOST_NAME} "cd ${CURRENT_PATH}; bin/getip.sh"`
+
+#Import cluster properties
+. conf/cluster.properties
+. conf/debugnc.properties
+
+#Clean up temp dir
+
+rm -rf $NCTMP_DIR2
+mkdir $NCTMP_DIR2
+
+#Clean up log dir
+rm -rf $NCLOGS_DIR2
+mkdir $NCLOGS_DIR2
+
+
+#Clean up I/O working dir
+io_dirs=$(echo $IO_DIRS2 | tr "," "\n")
+for io_dir in $io_dirs
+do
+	rm -rf $io_dir
+	mkdir $io_dir
+done
+
+#Set JAVA_HOME
+export JAVA_HOME=$JAVA_HOME
+
+#Get OS
+IPADDR=`bin/getip.sh`
+
+#Get node ID
+NODEID=`hostname | cut -d '.' -f 1`
+NODEID=${NODEID}2
+
+#Set JAVA_OPTS
+export JAVA_OPTS=$NCJAVA_OPTS2
+
+cd $HYRACKS_HOME
+HYRACKS_HOME=`pwd`
+
+#Enter the temp dir
+cd $NCTMP_DIR2
+
+#Launch hyracks nc
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS2}" &> $NCLOGS_DIR2/$NODEID.log &
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh b/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh
new file mode 100755
index 0000000..484ecac
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh
@@ -0,0 +1,31 @@
+#!/bin/bash
+hostname
+
+#Import cluster properties
+. conf/cluster.properties
+
+#Get the IP address of the cc
+CCHOST_NAME=`cat conf/master`
+CCHOST=`bin/getip.sh`
+
+#Remove the temp dir
+rm -rf $CCTMP_DIR
+mkdir $CCTMP_DIR
+
+#Remove the logs dir
+rm -rf $CCLOGS_DIR
+mkdir $CCLOGS_DIR
+
+#Export JAVA_HOME and JAVA_OPTS
+export JAVA_HOME=$JAVA_HOME
+export JAVA_OPTS=$CCJAVA_OPTS
+
+#Launch hyracks cc script
+chmod -R 755 $HYRACKS_HOME
+if [ -f "conf/topology.xml"  ]; then
+#Launch hyracks cc script with topology
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
+else
+#Launch hyracks cc script without toplogy
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
+fi
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh b/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh
new file mode 100755
index 0000000..23a4c36
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh
@@ -0,0 +1,49 @@
+hostname
+
+MY_NAME=`hostname`
+#Get the IP address of the cc
+CCHOST_NAME=`cat conf/master`
+CURRENT_PATH=`pwd`
+CCHOST=`ssh ${CCHOST_NAME} "cd ${CURRENT_PATH}; bin/getip.sh"`
+
+#Import cluster properties
+. conf/cluster.properties
+
+#Clean up temp dir
+
+rm -rf $NCTMP_DIR
+mkdir $NCTMP_DIR
+
+#Clean up log dir
+rm -rf $NCLOGS_DIR
+mkdir $NCLOGS_DIR
+
+
+#Clean up I/O working dir
+io_dirs=$(echo $IO_DIRS | tr "," "\n")
+for io_dir in $io_dirs
+do
+	rm -rf $io_dir
+	mkdir $io_dir
+done
+
+#Set JAVA_HOME
+export JAVA_HOME=$JAVA_HOME
+
+IPADDR=`bin/getip.sh`
+#echo $IPADDR
+
+#Get node ID
+NODEID=`hostname | cut -d '.' -f 1`
+
+#Set JAVA_OPTS
+export JAVA_OPTS=$NCJAVA_OPTS
+
+cd $HYRACKS_HOME
+HYRACKS_HOME=`pwd`
+
+#Enter the temp dir
+cd $NCTMP_DIR
+
+#Launch hyracks nc
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -result-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/stopAllNCs.sh b/hyracks/hyracks-dist/src/main/resources/bin/stopAllNCs.sh
new file mode 100755
index 0000000..12367c1
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/stopAllNCs.sh
@@ -0,0 +1,6 @@
+PREGELIX_PATH=`pwd`
+
+for i in `cat conf/slaves`
+do
+   ssh $i "cd ${PREGELIX_PATH}; bin/stopnc.sh"
+done
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/stopCluster.sh b/hyracks/hyracks-dist/src/main/resources/bin/stopCluster.sh
new file mode 100755
index 0000000..4889934
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/stopCluster.sh
@@ -0,0 +1,3 @@
+bin/stopAllNCs.sh
+sleep 2
+bin/stopcc.sh
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/stopcc.sh b/hyracks/hyracks-dist/src/main/resources/bin/stopcc.sh
new file mode 100755
index 0000000..c2f525a
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/stopcc.sh
@@ -0,0 +1,10 @@
+hostname
+. conf/cluster.properties
+
+#Kill process
+PID=`ps -ef|grep ${USER}|grep java|grep hyracks|awk '{print $2}'`
+echo $PID
+kill -9 $PID
+
+#Clean up CC temp dir
+rm -rf $CCTMP_DIR/*
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh b/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh
new file mode 100755
index 0000000..35c4794
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh
@@ -0,0 +1,27 @@
+hostname
+. conf/cluster.properties
+
+#Kill process
+PID=`ps -ef|grep ${USER}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
+
+if [ "$PID" == "" ]; then
+  PID=`ps -ef|grep ${USER}|grep java|grep 'hyracks'|awk '{print $2}'`
+fi
+
+if [ "$PID" == "" ]; then
+  USERID=`id | sed 's/^uid=//;s/(.*$//'`
+  PID=`ps -ef|grep ${USERID}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
+fi
+
+echo $PID
+kill -9 $PID
+
+#Clean up I/O working dir
+io_dirs=$(echo $IO_DIRS | tr "," "\n")
+for io_dir in $io_dirs
+do
+	rm -rf $io_dir/*
+done
+
+#Clean up NC temp dir
+rm -rf $NCTMP_DIR/*
diff --git a/hyracks/hyracks-dist/src/main/resources/conf/cluster.properties b/hyracks/hyracks-dist/src/main/resources/conf/cluster.properties
new file mode 100755
index 0000000..3b382f7
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/conf/cluster.properties
@@ -0,0 +1,37 @@
+#The CC port for Hyracks clients
+CC_CLIENTPORT=3099
+
+#The CC port for Hyracks cluster management
+CC_CLUSTERPORT=1099
+
+#The directory of hyracks binaries
+HYRACKS_HOME=../../../
+
+#The tmp directory for cc to install jars
+CCTMP_DIR=/tmp/t1
+
+#The tmp directory for nc to install jars
+NCTMP_DIR=/tmp/t2
+
+#The directory to put cc logs
+CCLOGS_DIR=$CCTMP_DIR/logs
+
+#The directory to put nc logs
+NCLOGS_DIR=$NCTMP_DIR/logs
+
+#Comma separated I/O directories for the spilling of external sort
+IO_DIRS="/tmp/t3,/tmp/t4"
+
+#The JAVA_HOME
+JAVA_HOME=$JAVA_HOME
+
+#The frame size of the internal dataflow engine
+FRAME_SIZE=65536
+
+#CC JAVA_OPTS
+CCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7001,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+# Yourkit option: -agentpath:/grid/0/dev/vborkar/tools/yjp-10.0.4/bin/linux-x86-64/libyjpagent.so=port=20001"
+
+#NC JAVA_OPTS
+NCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7002,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+
diff --git a/hyracks/hyracks-dist/src/main/resources/conf/debugnc.properties b/hyracks/hyracks-dist/src/main/resources/conf/debugnc.properties
new file mode 100755
index 0000000..27afa26
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/conf/debugnc.properties
@@ -0,0 +1,12 @@
+#The tmp directory for nc to install jars
+NCTMP_DIR2=/tmp/t-1
+
+#The directory to put nc logs
+NCLOGS_DIR2=$NCTMP_DIR/logs
+
+#Comma separated I/O directories for the spilling of external sort
+IO_DIRS2="/tmp/t-2,/tmp/t-3"
+
+#NC JAVA_OPTS
+NCJAVA_OPTS2="-Xdebug -Xrunjdwp:transport=dt_socket,address=7003,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+
diff --git a/hyracks/hyracks-dist/src/main/resources/conf/master b/hyracks/hyracks-dist/src/main/resources/conf/master
new file mode 100755
index 0000000..2fbb50c
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/conf/master
@@ -0,0 +1 @@
+localhost
diff --git a/hyracks/hyracks-dist/src/main/resources/conf/slaves b/hyracks/hyracks-dist/src/main/resources/conf/slaves
new file mode 100755
index 0000000..2fbb50c
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/conf/slaves
@@ -0,0 +1 @@
+localhost
diff --git a/hyracks/hyracks-dist/src/main/resources/conf/topology-template.xml b/hyracks/hyracks-dist/src/main/resources/conf/topology-template.xml
new file mode 100755
index 0000000..4710706
--- /dev/null
+++ b/hyracks/hyracks-dist/src/main/resources/conf/topology-template.xml
@@ -0,0 +1,7 @@
+<cluster-topology>
+    <network-switch name="Global">
+        <network-switch name="local">
+            <terminal name="127.0.0.1"/>
+        </network-switch>
+    </network-switch>
+</cluster-topology>
\ No newline at end of file
diff --git a/hyracks/hyracks-documentation/pom.xml b/hyracks/hyracks-documentation/pom.xml
index ed24adb..7aedd57 100644
--- a/hyracks/hyracks-documentation/pom.xml
+++ b/hyracks/hyracks-documentation/pom.xml
@@ -1,8 +1,6 @@
 <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-documentation</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hyracks-documentation</name>
 
   <parent>
diff --git a/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml b/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml
index 6350054..792cedd 100644
--- a/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml
+++ b/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml
@@ -2,7 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.btree</groupId>
   <artifactId>btreeapp</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>btreeapp</name>
 
   <parent>
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml b/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
index 82919ca..f941a5b 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
@@ -37,13 +37,15 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
             <configuration>
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml b/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml
index 8e90606..eb651ce 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml
@@ -41,8 +41,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
index 70c663d..4cf9745 100644
--- a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
+++ b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
@@ -2,7 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.compat</groupId>
   <artifactId>hadoopcompatapp</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hadoopcompatapp</name>
 
   <parent>
@@ -147,8 +146,9 @@
       	<artifactId>maven-compiler-plugin</artifactId>
       	<version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
       <plugin>
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
index 2f0b00f..f52536c 100644
--- a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
+++ b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
@@ -2,7 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.compat</groupId>
   <artifactId>hadoopcompatclient</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hadoopcompatclient</name>
 
   <parent>
@@ -33,13 +32,14 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
         </configuration>
       </plugin>
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
             <configuration>
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompathelper/pom.xml b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompathelper/pom.xml
index 1e029be..c397a72 100644
--- a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompathelper/pom.xml
+++ b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompathelper/pom.xml
@@ -2,7 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.compat</groupId>
   <artifactId>hadoopcompathelper</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hadoopcompathelper</name>
 
   <parent>
@@ -32,8 +31,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/pom.xml b/hyracks/hyracks-examples/hadoop-compat-example/pom.xml
index a2cf5ae..16787ca 100644
--- a/hyracks/hyracks-examples/hadoop-compat-example/pom.xml
+++ b/hyracks/hyracks-examples/hadoop-compat-example/pom.xml
@@ -2,7 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples</groupId>
   <artifactId>hadoop-compat-example</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>hadoop-compat-example</name>
 
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
index e31af75..5e7b5c9 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -16,8 +16,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
@@ -31,6 +32,13 @@
   		<scope>test</scope>
   	</dependency>
   	<dependency>
+        <groupId>edu.uci.ics.hyracks</groupId>
+        <artifactId>hyracks-client</artifactId>
+        <version>0.2.3-SNAPSHOT</version>
+        <type>jar</type>
+        <scope>compile</scope>
+    </dependency>
+  	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
   		<version>0.2.3-SNAPSHOT</version>
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
index 8482083..33ddca2 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,8 +45,8 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -60,6 +61,7 @@
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class BTreePrimaryIndexScanOperatorTest extends AbstractIntegrationTest {
     static {
@@ -114,7 +116,7 @@
         spec.addRoot(primaryCreateOp);
         runTest(spec);
     }
-    
+
     public void loadPrimaryIndexTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
@@ -143,8 +145,9 @@
 
         int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
         TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+                NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -179,13 +182,16 @@
         int[] highKeyFields = null; // + infinity
 
         BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
-                highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+                NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
index 82fecbe..acd3027 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,8 +45,8 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -60,6 +61,7 @@
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class BTreePrimaryIndexSearchOperatorTest extends AbstractIntegrationTest {
     static {
@@ -114,7 +116,7 @@
         spec.addRoot(primaryCreateOp);
         runTest(spec);
     }
-    
+
     public void loadPrimaryIndexTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
@@ -143,8 +145,9 @@
 
         int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
         TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+                NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -184,13 +187,16 @@
         int[] highKeyFields = { 1 };
 
         BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
-                highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+                NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
index e63ce11..ca03b16 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
@@ -29,6 +29,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -42,7 +43,7 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
@@ -57,6 +58,7 @@
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class BTreePrimaryIndexStatsOperatorTest extends AbstractIntegrationTest {
     static {
@@ -82,7 +84,6 @@
     private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
             new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
 
-
     @Before
     public void setup() throws Exception {
         // field, type and key declarations for primary index
@@ -107,7 +108,7 @@
         spec.addRoot(primaryCreateOp);
         runTest(spec);
     }
-    
+
     public void loadPrimaryIndexTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
@@ -136,8 +137,9 @@
 
         int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
         TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+                NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -153,12 +155,15 @@
         JobSpecification spec = new JobSpecification();
 
         TreeIndexStatsOperatorDescriptor primaryStatsOp = new TreeIndexStatsOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, primaryBtreeSplitProvider,
-                primaryTypeTraits, primaryComparatorFactories, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+                indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryStatsOp, NC1_ID);
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), primaryStatsOp, 0, printer, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
index 3c87ae3..34a1cd3 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,9 +45,9 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -63,6 +64,7 @@
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class BTreeSecondaryIndexInsertOperatorTest extends AbstractIntegrationTest {
     static {
@@ -339,9 +341,11 @@
                 primaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
index 1304f12..c9ee118 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,8 +45,8 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -60,6 +61,7 @@
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class BTreeSecondaryIndexSearchOperatorTest extends AbstractIntegrationTest {
     static {
@@ -137,7 +139,7 @@
         spec.addRoot(primaryCreateOp);
         runTest(spec);
     }
-    
+
     public void loadPrimaryIndexTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
@@ -166,8 +168,9 @@
 
         int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
         TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+                NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -180,14 +183,14 @@
 
     public void createSecondaryIndex() throws Exception {
         JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+        TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
+                storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryCreateOp, NC1_ID);
         spec.addRoot(secondaryCreateOp);
         runTest(spec);
     }
-    
+
     public void loadSecondaryIndexTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
@@ -212,8 +215,9 @@
 
         // scan primary index
         BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
-                highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+                NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
         // sort based on secondary keys
@@ -225,8 +229,9 @@
         // load secondary index
         int[] fieldPermutation = { 3, 0 };
         TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits, secondaryComparatorFactories, fieldPermutation, 0.7f,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+                NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeBulkLoad, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
@@ -268,8 +273,8 @@
         // search secondary index
         BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
                 secondaryRecDesc, storageManager, indexRegistryProvider, secondaryBtreeSplitProvider,
-                secondaryTypeTraits, secondaryComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
-                dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+                secondaryTypeTraits, secondaryComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true,
+                true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
 
         int[] primaryLowKeyFields = { 1 }; // second field from the tuples
@@ -279,13 +284,16 @@
 
         // search primary index
         BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, primaryLowKeyFields,
-                primaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, primaryLowKeyFields, primaryHighKeyFields, true, true,
+                dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index 023bdd9..8893567 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -14,29 +14,39 @@
  */
 package edu.uci.ics.hyracks.tests.integration;
 
+import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileReader;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.commons.io.FileUtils;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
 
 import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.client.dataset.HyracksDataset;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
 public abstract class AbstractIntegrationTest {
     private static final Logger LOGGER = Logger.getLogger(AbstractIntegrationTest.class.getName());
@@ -80,6 +90,7 @@
         ncConfig1.ccPort = 39001;
         ncConfig1.clusterNetIPAddress = "127.0.0.1";
         ncConfig1.dataIPAddress = "127.0.0.1";
+        ncConfig1.datasetIPAddress = "127.0.0.1";
         ncConfig1.nodeId = NC1_ID;
         nc1 = new NodeControllerService(ncConfig1);
         nc1.start();
@@ -89,6 +100,7 @@
         ncConfig2.ccPort = 39001;
         ncConfig2.clusterNetIPAddress = "127.0.0.1";
         ncConfig2.dataIPAddress = "127.0.0.1";
+        ncConfig2.datasetIPAddress = "127.0.0.1";
         ncConfig2.nodeId = NC2_ID;
         nc2 = new NodeControllerService(ncConfig2);
         nc2.start();
@@ -107,7 +119,7 @@
         cc.stop();
     }
 
-    protected void runTest(JobSpecification spec) throws Exception {
+    protected JobId executeTest(JobSpecification spec) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(spec.toJSON().toString(2));
         }
@@ -115,25 +127,72 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(jobId.toString());
         }
-        hcc.waitForCompletion(jobId);
-        dumpOutputFiles();
+        return jobId;
     }
 
-    private void dumpOutputFiles() {
-        if (LOGGER.isLoggable(Level.INFO)) {
-            for (File f : outputFiles) {
-                if (f.exists() && f.isFile()) {
-                    try {
-                        LOGGER.info("Reading file: " + f.getAbsolutePath() + " in test: " + getClass().getName());
-                        String data = FileUtils.readFileToString(f);
-                        LOGGER.info(data);
-                    } catch (IOException e) {
-                        LOGGER.info("Error reading file: " + f.getAbsolutePath());
-                        LOGGER.info(e.getMessage());
-                    }
+    protected void runTest(JobSpecification spec) throws Exception {
+        JobId jobId = executeTest(spec);
+        hcc.waitForCompletion(jobId);
+    }
+
+    protected List<String> readResults(JobSpecification spec, JobId jobId, ResultSetId resultSetId) throws Exception {
+        int nReaders = 1;
+        ByteBuffer resultBuffer = ByteBuffer.allocate(spec.getFrameSize());
+        resultBuffer.clear();
+
+        IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor(spec.getFrameSize());
+
+        IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
+        IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, resultSetId);
+
+        List<String> resultRecords = new ArrayList<String>();
+        ByteBufferInputStream bbis = new ByteBufferInputStream();
+
+        int readSize = reader.read(resultBuffer);
+
+        while (readSize > 0) {
+
+            try {
+                frameTupleAccessor.reset(resultBuffer);
+                for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+                    int start = frameTupleAccessor.getTupleStartOffset(tIndex);
+                    int length = frameTupleAccessor.getTupleEndOffset(tIndex) - start;
+                    bbis.setByteBuffer(resultBuffer, start);
+                    byte[] recordBytes = new byte[length];
+                    bbis.read(recordBytes, 0, length);
+                    resultRecords.add(new String(recordBytes, 0, length));
                 }
+            } finally {
+                bbis.close();
             }
+
+            resultBuffer.clear();
+            readSize = reader.read(resultBuffer);
         }
+        return resultRecords;
+    }
+
+    protected boolean runTestAndCompareResults(JobSpecification spec, String[] expectedFileNames) throws Exception {
+        JobId jobId = executeTest(spec);
+
+        List<String> results;
+        for (int i = 0; i < expectedFileNames.length; i++) {
+            results = readResults(spec, jobId, spec.getResultSetIds().get(i));
+            BufferedReader expectedFile = new BufferedReader(new FileReader(expectedFileNames[i]));
+
+            String expectedLine, actualLine;
+            int j = 0;
+            while ((expectedLine = expectedFile.readLine()) != null) {
+                actualLine = results.get(j).trim();
+                Assert.assertEquals(expectedLine, actualLine);
+                j++;
+            }
+            Assert.assertEquals(j, results.size());
+            expectedFile.close();
+        }
+
+        hcc.waitForCompletion(jobId);
+        return true;
     }
 
     protected File createTempFile() throws IOException {
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
index e0b8c73..24d0ef4 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
@@ -16,6 +16,7 @@
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
@@ -23,6 +24,7 @@
 import java.util.logging.Logger;
 
 import org.apache.commons.io.FileUtils;
+import org.json.JSONArray;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -30,13 +32,20 @@
 
 import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.client.dataset.HyracksDataset;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
 public abstract class AbstractMultiNCIntegrationTest {
 
@@ -84,6 +93,7 @@
             ncConfig.ccPort = 39001;
             ncConfig.clusterNetIPAddress = "127.0.0.1";
             ncConfig.dataIPAddress = "127.0.0.1";
+            ncConfig.datasetIPAddress = "127.0.0.1";
             ncConfig.nodeId = ASTERIX_IDS[i];
             asterixNCs[i] = new NodeControllerService(ncConfig);
             asterixNCs[i].start();
@@ -112,6 +122,46 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(jobId.toString());
         }
+
+        int nReaders = 1;
+
+        ByteBuffer resultBuffer = ByteBuffer.allocate(spec.getFrameSize());
+        resultBuffer.clear();
+
+        IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor(spec.getFrameSize());
+
+        IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
+        IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, spec.getResultSetIds().get(0));
+
+        JSONArray resultRecords = new JSONArray();
+        ByteBufferInputStream bbis = new ByteBufferInputStream();
+
+        int readSize = reader.read(resultBuffer);
+
+        while (readSize > 0) {
+
+            try {
+                frameTupleAccessor.reset(resultBuffer);
+                for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+                    int start = frameTupleAccessor.getTupleStartOffset(tIndex);
+                    int length = frameTupleAccessor.getTupleEndOffset(tIndex) - start;
+                    bbis.setByteBuffer(resultBuffer, start);
+                    byte[] recordBytes = new byte[length];
+                    bbis.read(recordBytes, 0, length);
+                    resultRecords.put(new String(recordBytes, 0, length));
+                }
+            } finally {
+                try {
+                    bbis.close();
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            resultBuffer.clear();
+            readSize = reader.read(resultBuffer);
+        }
+
         hcc.waitForCompletion(jobId);
         dumpOutputFiles();
     }
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
index 93e1e9b..06751e3 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
@@ -25,8 +25,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
@@ -49,7 +49,6 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableTableFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.AvgFieldGroupAggregatorFactory;
@@ -62,60 +61,42 @@
 import edu.uci.ics.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 /**
  *
  */
 public class AggregationTest extends AbstractIntegrationTest {
 
-    final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
-                    "data/tpch0.001/lineitem.tbl"))) });
+    final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC2_ID,
+            new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
 
-    final RecordDescriptor desc = new RecordDescriptor(
-            new ISerializerDeserializer[] {
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    IntegerSerializerDeserializer.INSTANCE,
-                    IntegerSerializerDeserializer.INSTANCE,
-                    IntegerSerializerDeserializer.INSTANCE,
-                    IntegerSerializerDeserializer.INSTANCE,
-                    FloatSerializerDeserializer.INSTANCE,
-                    FloatSerializerDeserializer.INSTANCE,
-                    FloatSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE });
+    final RecordDescriptor desc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+            FloatSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
-    final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(
-            new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                    IntegerParserFactory.INSTANCE,
-                    IntegerParserFactory.INSTANCE,
-                    IntegerParserFactory.INSTANCE,
-                    IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
-                    FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE, }, '|');
+    final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
+            UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+            IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
+            FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, }, '|');
 
-    private AbstractSingleActivityOperatorDescriptor getPrinter(
-            IOperatorDescriptorRegistry spec, String prefix) throws IOException {
+    private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification spec, String prefix)
+            throws IOException {
 
-        AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(
-                spec, new ConstantFileSplitProvider(new FileSplit[] {
-                        new FileSplit(NC1_ID, createTempFile()
-                                .getAbsolutePath()),
-                        new FileSplit(NC2_ID, createTempFile()
-                                .getAbsolutePath()) }), "\t");
+        ResultSetId rsId = new ResultSetId(1);
+        AbstractSingleActivityOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
 
         return printer;
     }
@@ -124,54 +105,38 @@
     public void singleKeySumInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new IntSumFieldAggregatorFactory(3, true),
-                                new FloatSumFieldAggregatorFactory(5, true) }),
-                outputRec, tableSize);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
+                        new FloatSumFieldAggregatorFactory(5, true) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -184,49 +149,34 @@
     public void singleKeySumPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE});
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new IntSumFieldAggregatorFactory(3, true),
-                                new FloatSumFieldAggregatorFactory(5, true)}),
-                outputRec);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
+                        new FloatSumFieldAggregatorFactory(5, true) }), outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -239,64 +189,43 @@
     public void singleKeySumExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE});
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
                                 new IntSumFieldAggregatorFactory(3, false),
-                                new FloatSumFieldAggregatorFactory(5, false)}),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
+                                new FloatSumFieldAggregatorFactory(5, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
                                 new IntSumFieldAggregatorFactory(2, false),
-                                new FloatSumFieldAggregatorFactory(3, false)}),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }),
-                        tableSize), true);
+                                new FloatSumFieldAggregatorFactory(3, false) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }), tableSize), true);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeySumExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -309,54 +238,38 @@
     public void singleKeyAvgInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new CountFieldAggregatorFactory(true),
-                                new AvgFieldGroupAggregatorFactory(1, true) }),
-                outputRec, tableSize);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -369,49 +282,34 @@
     public void singleKeyAvgPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new CountFieldAggregatorFactory(true),
-                                new AvgFieldGroupAggregatorFactory(1, true) }),
-                outputRec);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -424,64 +322,43 @@
     public void singleKeyAvgExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new CountFieldAggregatorFactory(false),
-                                new AvgFieldGroupAggregatorFactory(1, false) }),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
+                        new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
                                 new IntSumFieldAggregatorFactory(2, false),
-                                new AvgFieldMergeAggregatorFactory(3, false) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }),
-                        tableSize), true);
+                                new AvgFieldMergeAggregatorFactory(3, false) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }), tableSize), true);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -494,52 +371,38 @@
     public void singleKeyMinMaxStringInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, false) }), outputRec, tableSize);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true),
+                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -552,47 +415,34 @@
     public void singleKeyMinMaxStringPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, false) }), outputRec);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true),
+                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -605,63 +455,42 @@
     public void singleKeyMinMaxStringExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, true) }),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new MinMaxStringFieldAggregatorFactory(2, true,
-                                        true) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }),
-                        tableSize), true);
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new MinMaxStringFieldAggregatorFactory(15, true, true) }),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false),
+                        new MinMaxStringFieldAggregatorFactory(2, true, true) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }), tableSize), true);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -674,58 +503,39 @@
     public void multiKeySumInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec, keyFields, new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new IntSumFieldAggregatorFactory(3, true) }),
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
                 outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -738,51 +548,35 @@
     public void multiKeySumPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec, keyFields,
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new IntSumFieldAggregatorFactory(3, true) }),
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
                 outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -795,69 +589,43 @@
     public void multiKeySumExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new IntSumFieldAggregatorFactory(3, false) }),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(2, false),
-                                new IntSumFieldAggregatorFactory(3, false) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] {
-                                        PointableBinaryHashFunctionFactory
-                                                .of(UTF8StringPointable.FACTORY),
-                                        PointableBinaryHashFunctionFactory
-                                                .of(UTF8StringPointable.FACTORY) }),
-                        tableSize), true);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new IntSumFieldAggregatorFactory(3, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
+                                new IntSumFieldAggregatorFactory(3, false) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeySumExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -870,60 +638,40 @@
     public void multiKeyAvgInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec, keyFields, new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new CountFieldAggregatorFactory(true),
-                                new AvgFieldGroupAggregatorFactory(1, true) }),
-                outputRec, tableSize);
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -936,53 +684,36 @@
     public void multiKeyAvgPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec, keyFields,
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new CountFieldAggregatorFactory(true),
-                                new AvgFieldGroupAggregatorFactory(1, true) }),
-                outputRec);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -995,72 +726,46 @@
     public void multiKeyAvgExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new CountFieldAggregatorFactory(false),
-                                new AvgFieldGroupAggregatorFactory(1, false) }),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(2, false),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
+                        new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
                                 new IntSumFieldAggregatorFactory(3, false),
-                                new AvgFieldMergeAggregatorFactory(4, false) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] {
-                                        PointableBinaryHashFunctionFactory
-                                                .of(UTF8StringPointable.FACTORY),
-                                        PointableBinaryHashFunctionFactory
-                                                .of(UTF8StringPointable.FACTORY) }),
-                        tableSize), true);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
+                                new AvgFieldMergeAggregatorFactory(4, false) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyAvgExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -1073,58 +778,39 @@
     public void multiKeyMinMaxStringInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec, keyFields, new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, false) }), outputRec, tableSize);
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true),
+                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyMinMaxStringInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -1137,51 +823,35 @@
     public void multiKeyMinMaxStringPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec, keyFields,
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, false) }), outputRec);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true),
+                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyMinMaxStringPreClusterGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringPreClusterGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -1194,71 +864,44 @@
     public void multiKeyMinMaxStringExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, true) }),
-                new MultiFieldsAggregatorFactory(new int[] { 0, 1 },
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(2, false),
-                                new MinMaxStringFieldAggregatorFactory(3, true,
-                                        true) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] {
-                                        PointableBinaryHashFunctionFactory
-                                                .of(UTF8StringPointable.FACTORY),
-                                        PointableBinaryHashFunctionFactory
-                                                .of(UTF8StringPointable.FACTORY) }),
-                        tableSize), true);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new MinMaxStringFieldAggregatorFactory(15, true, true) }),
+                new MultiFieldsAggregatorFactory(new int[] { 0, 1 }, new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(2, false),
+                        new MinMaxStringFieldAggregatorFactory(3, true, true) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyMinMaxStringExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
index aea6126..5008991 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
@@ -25,6 +25,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,13 +45,14 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class CountOfCountsTest extends AbstractIntegrationTest {
     @Test
@@ -76,12 +78,10 @@
 
         RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                new int[] { 0 },
+        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                desc2);
+                new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID);
 
         InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
@@ -91,13 +91,15 @@
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(
                         new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
@@ -148,12 +150,10 @@
 
         RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                new int[] { 0 },
+        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                desc2);
+                new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
         InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
@@ -163,13 +163,16 @@
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(
                         new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
+
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
@@ -220,12 +223,10 @@
 
         RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                new int[] { 0 },
+        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                desc2);
+                new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
         InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
@@ -235,13 +236,16 @@
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(
                         new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
+
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
index 0d5a627..93ed2c7 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
@@ -26,8 +26,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
@@ -52,12 +52,13 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.FloatSumFieldAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class LocalityAwareConnectorTest extends AbstractMultiNCIntegrationTest {
 
@@ -200,13 +201,13 @@
         runTest(spec);
     }
 
-    private AbstractSingleActivityOperatorDescriptor getPrinter(IOperatorDescriptorRegistry spec, String prefix)
+    private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification spec, String prefix)
             throws IOException {
 
-        AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec,
-                new ConstantFileSplitProvider(new FileSplit[] {
-                        new FileSplit("asterix-005", createTempFile().getAbsolutePath()),
-                        new FileSplit("asterix-006", createTempFile().getAbsolutePath()) }), "\t");
+        ResultSetId rsId = new ResultSetId(1);
+        AbstractSingleActivityOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
 
         return printer;
     }
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
index 1ee4400..ec9be32 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -40,9 +41,10 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.LimitOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.OptimizedExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class OptimizedSortMergeTest extends AbstractIntegrationTest {
 
@@ -75,9 +77,11 @@
                         PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -128,9 +132,11 @@
         LimitOperatorDescriptor filter = new LimitOperatorDescriptor(spec, ordersDesc, outputLimit);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, filter, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
index 9355110..961f780 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
@@ -42,7 +43,8 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class ScanPrintTest extends AbstractIntegrationTest {
     @Test
@@ -63,10 +65,11 @@
                 desc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] {
-                new FileSplit(NC2_ID, createTempFile().getAbsolutePath()),
-                new FileSplit(NC1_ID, createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
+
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn = new OneToOneConnectorDescriptor(spec);
@@ -98,9 +101,11 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
+
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
@@ -135,9 +140,11 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
+
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
index 2c3fddf..0da93f2 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -40,9 +41,10 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class SortMergeTest extends AbstractIntegrationTest {
     @Test
@@ -73,9 +75,11 @@
                 ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -118,9 +122,11 @@
                         PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java
index 2b32142..6040748 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java
@@ -26,6 +26,7 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -35,8 +36,9 @@
 import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.LineFileWriteOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.SplitOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class SplitOperatorTest extends AbstractIntegrationTest {
 
@@ -50,6 +52,8 @@
             Assert.assertEquals(lineA, lineB);
         }
         Assert.assertNull(fileB.readLine());
+        fileA.close();
+        fileB.close();
     }
 
     @Test
@@ -83,8 +87,11 @@
 
         IOperatorDescriptor outputOp[] = new IOperatorDescriptor[outputFile.length];
         for (int i = 0; i < outputArity; i++) {
-            outputOp[i] = new LineFileWriteOperatorDescriptor(spec, new FileSplit[] { new FileSplit(NC1_ID,
-                    outputFile[i].getAbsolutePath()) });
+            ResultSetId rsId = new ResultSetId(i);
+            spec.addResultSetId(rsId);
+
+            outputOp[i] = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                    ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
             PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, outputOp[i], locations);
         }
 
@@ -96,10 +103,10 @@
         for (int i = 0; i < outputArity; i++) {
             spec.addRoot(outputOp[i]);
         }
-        runTest(spec);
-
+        String[] expectedResultsFileNames = new String[outputArity];
         for (int i = 0; i < outputArity; i++) {
-            compareFiles(inputFileName, outputFile[i].getAbsolutePath());
+            expectedResultsFileNames[i] = inputFileName;
         }
+        runTestAndCompareResults(spec, expectedResultsFileNames);
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index 61d4696..b5eb850 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -14,9 +14,7 @@
  */
 package edu.uci.ics.hyracks.tests.integration;
 
-import java.io.DataOutput;
 import java.io.File;
-import java.io.IOException;
 
 import org.junit.Test;
 
@@ -25,11 +23,10 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -47,35 +44,15 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.GraceHashJoinOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.HybridHashJoinOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.InMemoryHashJoinOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.MaterializingOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.NoopNullWriterFactory;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class TPCHCustomerOrderHashJoinTest extends AbstractIntegrationTest {
-    private static class NoopNullWriterFactory implements INullWriterFactory {
-
-        private static final long serialVersionUID = 1L;
-        public static final NoopNullWriterFactory INSTANCE = new NoopNullWriterFactory();
-
-        private NoopNullWriterFactory() {
-        }
-
-        @Override
-        public INullWriter createNullWriter() {
-            return new INullWriter() {
-                @Override
-                public void writeNull(DataOutput out) throws HyracksDataException {
-                    try {
-                        out.writeShort(0);
-                    } catch (IOException e) {
-                        throw new HyracksDataException(e);
-                    }
-                }
-            };
-        }
-    }
 
     /*
      * TPCH Customer table: CREATE TABLE CUSTOMER ( C_CUSTKEY INTEGER NOT NULL,
@@ -148,9 +125,11 @@
                 custOrderJoinDesc, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -229,9 +208,11 @@
                 custOrderJoinDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -310,9 +291,11 @@
                 custOrderJoinDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -392,9 +375,11 @@
                 custOrderJoinDesc, true, nullWriterFactories, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -478,9 +463,11 @@
                 custOrderJoinDesc, true, nullWriterFactories);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -564,9 +551,11 @@
                 custOrderJoinDesc, true, nullWriterFactories);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -643,9 +632,11 @@
                 custOrderJoinDesc, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -732,9 +723,11 @@
                 custOrderJoinDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -821,9 +814,11 @@
                 custOrderJoinDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -906,9 +901,11 @@
                 custOrderJoinDesc, 128);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -997,9 +994,11 @@
                 custOrderJoinDesc, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordPartConn = new MToNPartitioningConnectorDescriptor(spec,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
index 1e60372..99f2d18 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
@@ -25,10 +25,12 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -43,8 +45,10 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.NestedLoopJoinOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.NoopNullWriterFactory;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class TPCHCustomerOrderNestedLoopJoinTest extends AbstractIntegrationTest {
     private static class JoinComparatorFactory implements ITuplePairComparatorFactory {
@@ -165,12 +169,15 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
         NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 4);
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 4, false,
+                null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -239,12 +246,15 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
         NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 5);
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 5, false,
+                null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -313,12 +323,97 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
         NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 6);
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 6, false,
+                null);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
+        spec.connect(ordJoinConn, ordScanner, 0, join, 0);
+
+        IConnectorDescriptor custJoinConn = new MToNReplicatingConnectorDescriptor(spec);
+        spec.connect(custJoinConn, custScanner, 0, join, 1);
+
+        IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
+        spec.connect(joinPrinterConn, join, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Test
+    public void customerOrderCIDOuterJoinMulti() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] custSplits = new FileSplit[] {
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) };
+        IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
+        RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+        FileSplit[] ordersSplits = new FileSplit[] {
+                new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/orders-part1.tbl"))),
+                new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) };
+        IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
+        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
+
+        RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
+
+        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
+
+        FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
+
+        INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFieldCount()];
+        for (int j = 0; j < nullWriterFactories.length; j++) {
+            nullWriterFactories[j] = NoopNullWriterFactory.INSTANCE;
+        }
+
+        NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 5, true,
+                nullWriterFactories);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
+
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java
index 37b55b8..5b323e6 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -33,8 +34,9 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.union.UnionAllOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class UnionTest extends AbstractIntegrationTest {
     @Test
@@ -65,10 +67,11 @@
         UnionAllOperatorDescriptor unionAll = new UnionAllOperatorDescriptor(spec, 2, desc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, unionAll, NC2_ID, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] {
-                new FileSplit(NC2_ID, createTempFile().getAbsolutePath()),
-                new FileSplit(NC1_ID, createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), csvScanner01, 0, unionAll, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
index 836e72e..600b54b 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
@@ -8,6 +8,7 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
@@ -21,13 +22,14 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class BinaryTokenizerOperatorTest extends AbstractIntegrationTest {
 
@@ -58,9 +60,11 @@
                 tokenizerRecDesc, tokenizerFactory, tokenFields, keyFields);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
index 2206a26..b5a4df8 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
@@ -8,6 +8,7 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
@@ -21,13 +22,14 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class InvertedIndexOperatorsTest extends AbstractIntegrationTest {
 
@@ -58,9 +60,11 @@
                 tokenizerRecDesc, tokenizerFactory, tokenFields, projFields);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
index d8fd48e..d1071a3 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
@@ -48,8 +49,8 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -74,6 +75,7 @@
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class WordInvertedIndexTest extends AbstractIntegrationTest {
     static {
@@ -87,9 +89,12 @@
     private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
     private final static String sep = System.getProperty("file.separator");
     private final static String dateString = simpleDateFormat.format(new Date());
-    private final static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + "primaryBtree" + dateString;
-    private final static String btreeFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexBtree" + dateString;
-    private final static String invListsFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexLists" + dateString;
+    private final static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + "primaryBtree"
+            + dateString;
+    private final static String btreeFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexBtree"
+            + dateString;
+    private final static String invListsFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexLists"
+            + dateString;
 
     private IFileSplitProvider primaryFileSplitProvider = new ConstantFileSplitProvider(
             new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
@@ -155,7 +160,7 @@
         spec.addRoot(primaryCreateOp);
         runTest(spec);
     }
-    
+
     @Test
     public void testConjunctiveSearcher() throws Exception {
         IInvertedIndexSearchModifierFactory conjunctiveSearchModifierFactory = new ConjunctiveSearchModifierFactory();
@@ -180,8 +185,9 @@
     private IOperatorDescriptor createPrimaryBulkLoadOp(JobSpecification spec) {
         int[] fieldPermutation = { 0, 1 };
         TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
-                btreeDataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, fieldPermutation, 0.7f, btreeDataflowHelperFactory,
+                NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
         return primaryBtreeBulkLoad;
     }
@@ -206,8 +212,9 @@
         int[] lowKeyFields = null; // - infinity
         int[] highKeyFields = null; // + infinity
         BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
-                highKeyFields, true, true, btreeDataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, btreeDataflowHelperFactory, false,
+                NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
         return primaryBtreeSearchOp;
     }
@@ -227,9 +234,12 @@
         JobSpecification spec = new JobSpecification();
         IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
         IOperatorDescriptor primaryScanOp = createPrimaryScanOp(spec);
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, printer, 0);
         spec.addRoot(printer);
@@ -265,15 +275,14 @@
     public void createInvertedIndex() throws Exception {
         JobSpecification spec = new JobSpecification();
         InvertedIndexCreateOperatorDescriptor invIndexCreateOp = new InvertedIndexCreateOperatorDescriptor(spec,
-                storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
-                indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
-                invListsComparatorFactories, tokenizerFactory, btreeDataflowHelperFactory,
-                NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, btreeFileSplitProvider, invListsFileSplitProvider, indexRegistryProvider,
+                tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
+                tokenizerFactory, btreeDataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexCreateOp, NC1_ID);
         spec.addRoot(invIndexCreateOp);
         runTest(spec);
     }
-    
+
     public void loadInvertedIndex() throws Exception {
         JobSpecification spec = new JobSpecification();
         IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
@@ -325,20 +334,23 @@
         JobSpecification spec = new JobSpecification();
         IOperatorDescriptor queryProviderOp = createQueryProviderOp(spec, queryString);
         IOperatorDescriptor invIndexSearchOp = createInvertedIndexSearchOp(spec, searchModifierFactory);
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         spec.connect(new OneToOneConnectorDescriptor(spec), queryProviderOp, 0, invIndexSearchOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), invIndexSearchOp, 0, printer, 0);
         spec.addRoot(printer);
         runTest(spec);
     }
-    
+
     @AfterClass
     public static void cleanup() throws Exception {
-    	File primary = new File(primaryFileName);
-    	File btree = new File(btreeFileName);
-    	File invLists = new File(invListsFileName);
+        File primary = new File(primaryFileName);
+        File btree = new File(btreeFileName);
+        File invLists = new File(invListsFileName);
         primary.deleteOnExit();
         btree.deleteOnExit();
         invLists.deleteOnExit();
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
index 6625148..92b6e14 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -47,8 +48,8 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -64,6 +65,7 @@
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class RTreePrimaryIndexSearchOperatorTest extends AbstractIntegrationTest {
     static {
@@ -190,9 +192,11 @@
                 dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryRTreeSearchOp, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryRTreeSearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
index ef2950e..2a00394 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
@@ -29,6 +29,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -45,6 +46,7 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
@@ -64,6 +66,7 @@
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class RTreePrimaryIndexStatsOperatorTest extends AbstractIntegrationTest {
     static {
@@ -175,9 +178,11 @@
                 primaryTypeTraits, primaryComparatorFactories, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryStatsOp, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), primaryStatsOp, 0, printer, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index 030afcf..1d86037 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -47,8 +48,8 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -67,6 +68,7 @@
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class RTreeSecondaryIndexSearchOperatorTest extends AbstractIntegrationTest {
     static {
@@ -297,9 +299,11 @@
                 secondaryTypeTraits, secondaryComparatorFactories, keyFields, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryRTreeSearchOp, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryRTreeSearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/NoopNullWriterFactory.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/NoopNullWriterFactory.java
new file mode 100644
index 0000000..d119509
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/NoopNullWriterFactory.java
@@ -0,0 +1,31 @@
+package edu.uci.ics.hyracks.tests.util;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class NoopNullWriterFactory implements INullWriterFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static final NoopNullWriterFactory INSTANCE = new NoopNullWriterFactory();
+
+    private NoopNullWriterFactory() {
+    }
+
+    @Override
+    public INullWriter createNullWriter() {
+        return new INullWriter() {
+            @Override
+            public void writeNull(DataOutput out) throws HyracksDataException {
+                try {
+                    out.writeShort(0);
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        };
+    }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/ResultSerializerFactoryProvider.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/ResultSerializerFactoryProvider.java
new file mode 100644
index 0000000..19c4475
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/ResultSerializerFactoryProvider.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.tests.util;
+
+import java.io.DataInputStream;
+import java.io.PrintStream;
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+public class ResultSerializerFactoryProvider implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public static final ResultSerializerFactoryProvider INSTANCE = new ResultSerializerFactoryProvider();
+
+    private ResultSerializerFactoryProvider() {
+    }
+
+    public IResultSerializerFactory getResultSerializerFactoryProvider() {
+        return new IResultSerializerFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IResultSerializer createResultSerializer(final RecordDescriptor recordDesc,
+                    final PrintStream printStream) {
+                return new IResultSerializer() {
+                    private static final long serialVersionUID = 1L;
+
+                    ByteBufferInputStream bbis = new ByteBufferInputStream();
+                    DataInputStream di = new DataInputStream(bbis);
+
+                    @Override
+                    public void init() throws HyracksDataException {
+
+                    }
+
+                    @Override
+                    public boolean appendTuple(IFrameTupleAccessor tAccess, int tIdx) throws HyracksDataException {
+                        int start = tAccess.getTupleStartOffset(tIdx) + tAccess.getFieldSlotsLength();
+
+                        bbis.setByteBuffer(tAccess.getBuffer(), start);
+
+                        Object[] record = new Object[recordDesc.getFieldCount()];
+                        for (int i = 0; i < record.length; ++i) {
+                            Object instance = recordDesc.getFields()[i].deserialize(di);
+                            if (i == 0) {
+                                printStream.print(String.valueOf(instance));
+                            } else {
+                                printStream.print(", " + String.valueOf(instance));
+                            }
+                        }
+                        printStream.println();
+                        return true;
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/hyracks/hyracks-examples/pom.xml b/hyracks/hyracks-examples/pom.xml
index 8ce8108..551e2be 100644
--- a/hyracks/hyracks-examples/pom.xml
+++ b/hyracks/hyracks-examples/pom.xml
@@ -1,8 +1,6 @@
 <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-examples</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>hyracks-examples</name>
 
diff --git a/hyracks/hyracks-examples/text-example/pom.xml b/hyracks/hyracks-examples/text-example/pom.xml
index ba8649e..0476bb3 100644
--- a/hyracks/hyracks-examples/text-example/pom.xml
+++ b/hyracks/hyracks-examples/text-example/pom.xml
@@ -2,7 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples</groupId>
   <artifactId>text-example</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>text-example</name>
 
diff --git a/hyracks/hyracks-examples/text-example/textapp/pom.xml b/hyracks/hyracks-examples/text-example/textapp/pom.xml
index 8ac69d8..945df0b 100644
--- a/hyracks/hyracks-examples/text-example/textapp/pom.xml
+++ b/hyracks/hyracks-examples/text-example/textapp/pom.xml
@@ -2,7 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.text</groupId>
   <artifactId>textapp</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>textapp</name>
 
   <parent>
@@ -142,8 +141,9 @@
       	<artifactId>maven-compiler-plugin</artifactId>
       	<version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
       <plugin>
diff --git a/hyracks/hyracks-examples/text-example/textclient/pom.xml b/hyracks/hyracks-examples/text-example/textclient/pom.xml
index d593ef9..4aace73 100644
--- a/hyracks/hyracks-examples/text-example/textclient/pom.xml
+++ b/hyracks/hyracks-examples/text-example/textclient/pom.xml
@@ -2,7 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.text</groupId>
   <artifactId>textclient</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>textclient</name>
 
   <parent>
@@ -33,13 +32,15 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
           <id>textclient</id>
diff --git a/hyracks/hyracks-examples/text-example/texthelper/pom.xml b/hyracks/hyracks-examples/text-example/texthelper/pom.xml
index 8e32c8c..bcb280c 100644
--- a/hyracks/hyracks-examples/text-example/texthelper/pom.xml
+++ b/hyracks/hyracks-examples/text-example/texthelper/pom.xml
@@ -36,8 +36,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-examples/tpch-example/pom.xml b/hyracks/hyracks-examples/tpch-example/pom.xml
index b237c9b..9951792 100644
--- a/hyracks/hyracks-examples/tpch-example/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/pom.xml
@@ -2,7 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples</groupId>
   <artifactId>tpch-example</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>tpch-example</name>
 
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
index 3170306..4e0d9f0 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -29,13 +29,15 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
             <configuration>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index 01ccdef..0ad0ff0 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -199,7 +199,7 @@
 
         if ("nestedloop".equalsIgnoreCase(algo)) {
             join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                    PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), custOrderJoinDesc, memSize);
+                    PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), custOrderJoinDesc, memSize, false, null);
 
         } else if ("gracehash".equalsIgnoreCase(algo)) {
             join = new GraceHashJoinOperatorDescriptor(
diff --git a/hyracks/hyracks-hadoop-compat/pom.xml b/hyracks/hyracks-hadoop-compat/pom.xml
index 849efe3..87aaaa7 100644
--- a/hyracks/hyracks-hadoop-compat/pom.xml
+++ b/hyracks/hyracks-hadoop-compat/pom.xml
@@ -18,13 +18,15 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
             <configuration>
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/pom.xml b/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/pom.xml
new file mode 100644
index 0000000..9092655
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/pom.xml
@@ -0,0 +1,108 @@
+<?xml version="1.0"?>
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+	<modelVersion>4.0.0</modelVersion>
+	<artifactId>hyracks-hdfs-0.20.2</artifactId>
+	<name>hyracks-hdfs-0.20.2</name>
+	<parent>
+		<artifactId>hyracks-hdfs</artifactId>
+		<groupId>edu.uci.ics.hyracks</groupId>
+		<version>0.2.3-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.7</source>
+					<target>1.7</target>
+					<fork>true</fork>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<version>2.7.2</version>
+				<configuration>
+					<forkMode>pertest</forkMode>
+					<includes>
+						<include>**/*TestSuite.java</include>
+						<include>**/*Test.java</include>
+					</includes>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+
+	<profiles>
+		<profile>
+			<activation>
+				<activeByDefault>true</activeByDefault>
+			</activation>
+			<id>hadoop-0.20.2</id>
+			<dependencies>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-core</artifactId>
+					<version>0.20.2</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-test</artifactId>
+					<version>0.20.2</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+		<profile>
+			<activation>
+				<activeByDefault>false</activeByDefault>
+				<property>
+					<name>hadoop</name>
+					<value>1.0.4</value>
+				</property>
+			</activation>
+			<id>hadoop-1.0.4</id>
+			<dependencies>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-core</artifactId>
+					<version>1.0.4</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-minicluster</artifactId>
+					<version>1.0.4</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-test</artifactId>
+					<version>1.0.4</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+	</profiles>
+
+	<dependencies>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-api</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+	</dependencies>
+</project>
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/src/main/java/edu/uci/ics/hyracks/hdfs/ContextFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/src/main/java/edu/uci/ics/hyracks/hdfs/ContextFactory.java
new file mode 100644
index 0000000..16ce76b
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/src/main/java/edu/uci/ics/hyracks/hdfs/ContextFactory.java
@@ -0,0 +1,39 @@
+package edu.uci.ics.hyracks.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * The wrapper to generate TaskTattemptContext
+ */
+public class ContextFactory {
+
+    @SuppressWarnings({ "unchecked", "rawtypes" })
+    public TaskAttemptContext createContext(Configuration conf, TaskAttemptID tid) throws HyracksDataException {
+        try {
+            return new Mapper().new Context(conf, tid, null, null, null, null, null);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public TaskAttemptContext createContext(Configuration conf, int partition) throws HyracksDataException {
+        try {
+            TaskAttemptID tid = new TaskAttemptID("", 0, true, partition, 0);
+            return new TaskAttemptContext(conf, tid);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public JobContext createJobContext(Configuration conf) {
+        return new JobContext(conf, new JobID("0", 0));
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/src/main/java/edu/uci/ics/hyracks/hdfs/MiniDFSClusterFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/src/main/java/edu/uci/ics/hyracks/hdfs/MiniDFSClusterFactory.java
new file mode 100644
index 0000000..9133d35
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/src/main/java/edu/uci/ics/hyracks/hdfs/MiniDFSClusterFactory.java
@@ -0,0 +1,17 @@
+package edu.uci.ics.hyracks.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class MiniDFSClusterFactory {
+
+    public MiniDFSCluster getMiniDFSCluster(Configuration conf, int numberOfNC) throws HyracksDataException {
+        try {
+            return new MiniDFSCluster(conf, numberOfNC, true, null);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/pom.xml b/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/pom.xml
new file mode 100644
index 0000000..8b7ecf0
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/pom.xml
@@ -0,0 +1,219 @@
+<?xml version="1.0"?>
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+	<modelVersion>4.0.0</modelVersion>
+	<artifactId>hyracks-hdfs-0.23.1</artifactId>
+	<name>hyracks-hdfs-0.23.1</name>
+	<parent>
+		<artifactId>hyracks-hdfs</artifactId>
+		<groupId>edu.uci.ics.hyracks</groupId>
+		<version>0.2.3-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.7</source>
+					<target>1.7</target>
+					<fork>true</fork>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<version>2.7.2</version>
+				<configuration>
+					<forkMode>pertest</forkMode>
+					<includes>
+						<include>**/*TestSuite.java</include>
+						<include>**/*Test.java</include>
+					</includes>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+
+	<profiles>
+		<profile>
+			<activation>
+				<activeByDefault>true</activeByDefault>
+				<property>
+					<name>hadoop</name>
+					<value>0.23.1</value>
+				</property>
+			</activation>
+			<id>hadoop-0.23.1</id>
+			<dependencies>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-common</artifactId>
+					<version>0.23.1</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-mapreduce-client-core</artifactId>
+					<version>0.23.1</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-hdfs</artifactId>
+					<version>0.23.1</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-minicluster</artifactId>
+					<version>0.23.1</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+		<profile>
+			<id>hadoop-0.23.6</id>
+			<activation>
+				<activeByDefault>false</activeByDefault>
+				<property>
+					<name>hadoop</name>
+					<value>0.23.6</value>
+				</property>
+			</activation>
+			<dependencies>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-common</artifactId>
+					<version>0.23.6</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-mapreduce-client-core</artifactId>
+					<version>0.23.6</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-hdfs</artifactId>
+					<version>0.23.6</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-minicluster</artifactId>
+					<version>0.23.6</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+		<profile>
+			<activation>
+				<activeByDefault>false</activeByDefault>
+				<property>
+					<name>hadoop</name>
+					<value>cdh-4.2</value>
+				</property>
+			</activation>
+			<id>cdh-4.2</id>
+			<dependencies>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-common</artifactId>
+					<version>2.0.0-cdh4.2.0</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-mapreduce-client-core</artifactId>
+					<version>2.0.0-cdh4.2.0</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-hdfs</artifactId>
+					<version>2.0.0-cdh4.2.0</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-minicluster</artifactId>
+					<version>2.0.0-cdh4.2.0</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+		<profile>
+			<activation>
+				<activeByDefault>false</activeByDefault>
+				<property>
+					<name>hadoop</name>
+					<value>cdh-4.1</value>
+				</property>
+			</activation>
+			<id>cdh-4.1</id>
+			<dependencies>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-common</artifactId>
+					<version>2.0.0-cdh4.1.0</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-mapreduce-client-core</artifactId>
+					<version>2.0.0-cdh4.1.0</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-hdfs</artifactId>
+					<version>2.0.0-cdh4.1.0</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-minicluster</artifactId>
+					<version>2.0.0-cdh4.1.0</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+	</profiles>
+
+	<dependencies>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-api</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+	</dependencies>
+
+	<repositories>
+		<repository>
+			<id>cloudera</id>
+			<url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
+		</repository>
+	</repositories>
+</project>
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/src/main/java/edu/uci/ics/hyracks/hdfs/ContextFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/src/main/java/edu/uci/ics/hyracks/hdfs/ContextFactory.java
new file mode 100644
index 0000000..ddcce64
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/src/main/java/edu/uci/ics/hyracks/hdfs/ContextFactory.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hyracks.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * The wrapper to generate TaskTattemptContext
+ */
+public class ContextFactory {
+
+    public TaskAttemptContext createContext(Configuration conf, TaskAttemptID tid) throws HyracksDataException {
+        try {
+            return new TaskAttemptContextImpl(conf, tid);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public TaskAttemptContext createContext(Configuration conf, int partition) throws HyracksDataException {
+        try {
+            TaskAttemptID tid = new TaskAttemptID("", 0, TaskType.REDUCE, partition, 0);
+            return new TaskAttemptContextImpl(conf, tid);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public JobContext createJobContext(Configuration conf) {
+        return new JobContextImpl(conf, new JobID("0", 0));
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/src/main/java/edu/uci/ics/hyracks/hdfs/MiniDFSClusterFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/src/main/java/edu/uci/ics/hyracks/hdfs/MiniDFSClusterFactory.java
new file mode 100644
index 0000000..ded75f1
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/src/main/java/edu/uci/ics/hyracks/hdfs/MiniDFSClusterFactory.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.hyracks.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class MiniDFSClusterFactory {
+
+    public MiniDFSCluster getMiniDFSCluster(Configuration conf, int numberOfNC) throws HyracksDataException {
+        try {
+            MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+            builder.numDataNodes(numberOfNC);
+            MiniDFSCluster dfsCluster = builder.build();
+            return dfsCluster;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml b/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml
new file mode 100644
index 0000000..a28c698a
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml
@@ -0,0 +1,217 @@
+<?xml version="1.0"?>
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+	<modelVersion>4.0.0</modelVersion>
+	<artifactId>hyracks-hdfs-core</artifactId>
+	<name>hyracks-hdfs-core</name>
+	<parent>
+		<artifactId>hyracks-hdfs</artifactId>
+		<groupId>edu.uci.ics.hyracks</groupId>
+		<version>0.2.3-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.7</source>
+					<target>1.7</target>
+					<fork>true</fork>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<version>2.7.2</version>
+				<configuration>
+					<forkMode>pertest</forkMode>
+					<includes>
+						<include>**/*TestSuite.java</include>
+						<include>**/*Test.java</include>
+					</includes>
+				</configuration>
+			</plugin>
+			<plugin>
+				<artifactId>maven-clean-plugin</artifactId>
+				<version>2.5</version>
+				<configuration>
+					<filesets>
+						<fileset>
+							<directory>.</directory>
+							<includes>
+								<include>edu*</include>
+								<include>actual*</include>
+								<include>build*</include>
+								<include>expect*</include>
+								<include>ClusterController*</include>
+								<include>edu.uci.*</include>
+							</includes>
+						</fileset>
+					</filesets>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+
+	<profiles>
+		<profile>
+			<activation>
+				<activeByDefault>true</activeByDefault>
+			</activation>
+			<id>hadoop-0.20.2</id>
+			<dependencies>
+				<dependency>
+					<groupId>edu.uci.ics.hyracks</groupId>
+					<artifactId>hyracks-hdfs-0.20.2</artifactId>
+					<version>0.2.3-SNAPSHOT</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+		<profile>
+			<activation>
+				<activeByDefault>false</activeByDefault>
+				<property>
+					<name>hadoop</name>
+					<value>1.0.4</value>
+				</property>
+			</activation>
+			<id>hadoop-1.0.4</id>
+			<dependencies>
+				<dependency>
+					<groupId>edu.uci.ics.hyracks</groupId>
+					<artifactId>hyracks-hdfs-0.20.2</artifactId>
+					<version>0.2.3-SNAPSHOT</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+		<profile>
+			<activation>
+				<activeByDefault>false</activeByDefault>
+				<property>
+					<name>hadoop</name>
+					<value>0.23.1</value>
+				</property>
+			</activation>
+			<id>hadoop-0.23.1</id>
+			<dependencies>
+				<dependency>
+					<groupId>edu.uci.ics.hyracks</groupId>
+					<artifactId>hyracks-hdfs-0.23.1</artifactId>
+					<version>0.2.3-SNAPSHOT</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+		<profile>
+			<activation>
+				<activeByDefault>false</activeByDefault>
+				<property>
+					<name>hadoop</name>
+					<value>0.23.6</value>
+				</property>
+			</activation>
+			<id>hadoop-0.23.6</id>
+			<dependencies>
+				<dependency>
+					<groupId>edu.uci.ics.hyracks</groupId>
+					<artifactId>hyracks-hdfs-0.23.1</artifactId>
+					<version>0.2.3-SNAPSHOT</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+		<profile>
+			<activation>
+				<activeByDefault>false</activeByDefault>
+				<property>
+					<name>hadoop</name>
+					<value>cdh-4.1</value>
+				</property>
+			</activation>
+			<id>cdh-4.1</id>
+			<dependencies>
+				<dependency>
+					<groupId>edu.uci.ics.hyracks</groupId>
+					<artifactId>hyracks-hdfs-0.23.1</artifactId>
+					<version>0.2.3-SNAPSHOT</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+		<profile>
+			<activation>
+				<activeByDefault>false</activeByDefault>
+				<property>
+					<name>hadoop</name>
+					<value>cdh-4.2</value>
+				</property>
+			</activation>
+			<id>cdh-4.2</id>
+			<dependencies>
+				<dependency>
+					<groupId>edu.uci.ics.hyracks</groupId>
+					<artifactId>hyracks-hdfs-0.23.1</artifactId>
+					<version>0.2.3-SNAPSHOT</version>
+					<type>jar</type>
+					<scope>compile</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+	</profiles>
+
+	<dependencies>
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<version>3.8.1</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-api</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-dataflow-std</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-dataflow-common</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-control-cc</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-control-nc</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.commons</groupId>
+			<artifactId>commons-io</artifactId>
+			<version>1.3.2</version>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+</project>
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParser.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParser.java
new file mode 100644
index 0000000..5d35ec5
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParser.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2012 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.hdfs.api;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Users need to implement this interface to use the HDFSReadOperatorDescriptor.
+ * 
+ * @param <K>
+ *            the key type
+ * @param <V>
+ *            the value type
+ */
+public interface IKeyValueParser<K, V> {
+
+    /**
+     * Initialize the key value parser.
+     * 
+     * @param writer
+     *            The hyracks writer for outputting data.
+     * @throws HyracksDataException
+     */
+    public void open(IFrameWriter writer) throws HyracksDataException;
+
+    /**
+     * Parse a key-value pair returned by HDFS record reader to a tuple.
+     * when the parsers' internal buffer is full, it can flush the buffer to the writer
+     * 
+     * @param key
+     *            The key returned from Hadoop's InputReader.
+     * @param value
+     *            The value returned from Hadoop's InputReader.
+     * @param writer
+     *            The hyracks writer for outputting data.
+     * @throws HyracksDataException
+     */
+    public void parse(K key, V value, IFrameWriter writer) throws HyracksDataException;
+
+    /**
+     * Flush the residual tuples in the internal buffer to the writer.
+     * This method is called in the close() of HDFSReadOperatorDescriptor.
+     * 
+     * @param writer
+     *            The hyracks writer for outputting data.
+     * @throws HyracksDataException
+     */
+    public void close(IFrameWriter writer) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParserFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParserFactory.java
new file mode 100644
index 0000000..7d6f868
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParserFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2012 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.hdfs.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Users need to implement this interface to use the HDFSReadOperatorDescriptor.
+ * 
+ * @param <K>
+ *            the key type
+ * @param <V>
+ *            the value type
+ */
+public interface IKeyValueParserFactory<K, V> extends Serializable {
+
+    /**
+     * This method creates a key-value parser.
+     * 
+     * @param ctx
+     *            the IHyracksTaskContext
+     * @return a key-value parser instance.
+     */
+    public IKeyValueParser<K, V> createKeyValueParser(IHyracksTaskContext ctx) throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollection.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollection.java
new file mode 100644
index 0000000..c51c1dd
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollection.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.hyracks.hdfs.api;
+
+import org.apache.hadoop.mapred.InputSplit;
+
+@SuppressWarnings("deprecation")
+public interface INcCollection {
+
+    public String findNearestAvailableSlot(InputSplit split);
+
+    public int numAvailableSlots();
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollectionBuilder.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollectionBuilder.java
new file mode 100644
index 0000000..ef3ff23
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollectionBuilder.java
@@ -0,0 +1,18 @@
+package edu.uci.ics.hyracks.hdfs.api;
+
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+
+/**
+ * NC collections
+ * 
+ * @author yingyib
+ */
+public interface INcCollectionBuilder {
+
+    public INcCollection build(Map<String, NodeControllerInfo> ncNameToNcInfos,
+            Map<String, List<String>> ipToNcMapping, Map<String, Integer> ncNameToIndex, String[] NCs, int[] workloads,
+            int slotLimit);
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriter.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriter.java
new file mode 100644
index 0000000..8e85627
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriter.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2012 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.hdfs.api;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Users need to implement this interface to use the HDFSWriteOperatorDescriptor.
+ */
+public interface ITupleWriter {
+
+    /**
+     * Initialize the the tuple writer.
+     * 
+     * @param output
+     *            The channel for output data.
+     * @throws HyracksDataException
+     */
+    public void open(DataOutput output) throws HyracksDataException;
+
+    /**
+     * Write the tuple to the DataOutput.
+     * 
+     * @param output
+     *            the DataOutput channel
+     * @param tuple
+     *            the tuple to write
+     * @throws HyracksDataException
+     */
+    public void write(DataOutput output, ITupleReference tuple) throws HyracksDataException;
+
+    /**
+     * Close the writer.
+     * 
+     * @param output
+     *            The channel for output data.
+     * @throws HyracksDataException
+     */
+    public void close(DataOutput output) throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriterFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriterFactory.java
new file mode 100644
index 0000000..9a025c2
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriterFactory.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2012 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.hdfs.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Users need to implement this interface to use the HDFSWriteOperatorDescriptor.
+ */
+public interface ITupleWriterFactory extends Serializable {
+
+    /**
+     * @param ctx
+     *            the IHyracksTaskContext
+     * @return a tuple writer instance
+     */
+    public ITupleWriter getTupleWriter(IHyracksTaskContext ctx) throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/ConfFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/ConfFactory.java
new file mode 100644
index 0000000..4fa0164
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/ConfFactory.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.hyracks.hdfs.dataflow;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.Serializable;
+
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings("deprecation")
+public class ConfFactory implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private byte[] confBytes;
+
+    public ConfFactory(JobConf conf) throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutputStream dos = new DataOutputStream(bos);
+            conf.write(dos);
+            confBytes = bos.toByteArray();
+            dos.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public JobConf getConf() throws HyracksDataException {
+        try {
+            JobConf conf = new JobConf();
+            DataInputStream dis = new DataInputStream(new ByteArrayInputStream(confBytes));
+            conf.readFields(dis);
+            dis.close();
+            return conf;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java
new file mode 100644
index 0000000..f49688b
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java
@@ -0,0 +1,145 @@
+/*
+ * Copyright 2009-2012 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.hdfs.dataflow;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+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.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParser;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParserFactory;
+
+/**
+ * The HDFS file read operator using the Hadoop old API.
+ * To use this operator, a user need to provide an IKeyValueParserFactory implementation which convert
+ * key-value pairs into tuples.
+ */
+@SuppressWarnings({ "deprecation", "rawtypes" })
+public class HDFSReadOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final ConfFactory confFactory;
+    private final InputSplitsFactory splitsFactory;
+    private final String[] scheduledLocations;
+    private final IKeyValueParserFactory tupleParserFactory;
+    private final boolean[] executed;
+
+    /**
+     * The constructor of HDFSReadOperatorDescriptor.
+     * 
+     * @param spec
+     *            the JobSpecification object
+     * @param rd
+     *            the output record descriptor
+     * @param conf
+     *            the Hadoop JobConf object, which contains the input format and the input paths
+     * @param splits
+     *            the array of FileSplits (HDFS chunks).
+     * @param scheduledLocations
+     *            the node controller names to scan the FileSplits, which is an one-to-one mapping. The String array
+     *            is obtained from the edu.cui.ics.hyracks.hdfs.scheduler.Scheduler.getLocationConstraints(InputSplits[]).
+     * @param tupleParserFactory
+     *            the ITupleParserFactory implementation instance.
+     * @throws HyracksException
+     */
+    public HDFSReadOperatorDescriptor(JobSpecification spec, RecordDescriptor rd, JobConf conf, InputSplit[] splits,
+            String[] scheduledLocations, IKeyValueParserFactory tupleParserFactory) throws HyracksException {
+        super(spec, 0, 1);
+        try {
+            this.splitsFactory = new InputSplitsFactory(splits);
+            this.confFactory = new ConfFactory(conf);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+        this.scheduledLocations = scheduledLocations;
+        this.executed = new boolean[scheduledLocations.length];
+        Arrays.fill(executed, false);
+        this.tupleParserFactory = tupleParserFactory;
+        this.recordDescriptors[0] = rd;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+            throws HyracksDataException {
+        final InputSplit[] inputSplits = splitsFactory.getSplits();
+
+        return new AbstractUnaryOutputSourceOperatorNodePushable() {
+            private String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+
+            @SuppressWarnings("unchecked")
+            @Override
+            public void initialize() throws HyracksDataException {
+                ClassLoader ctxCL = Thread.currentThread().getContextClassLoader();
+                try {
+                    Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
+                    JobConf conf = confFactory.getConf();
+                    IKeyValueParser parser = tupleParserFactory.createKeyValueParser(ctx);
+                    writer.open();
+                    parser.open(writer);
+                    InputFormat inputFormat = conf.getInputFormat();
+                    for (int i = 0; i < inputSplits.length; i++) {
+                        /**
+                         * read all the partitions scheduled to the current node
+                         */
+                        if (scheduledLocations[i].equals(nodeName)) {
+                            /**
+                             * pick an unread split to read
+                             * synchronize among simultaneous partitions in the same machine
+                             */
+                            synchronized (executed) {
+                                if (executed[i] == false) {
+                                    executed[i] = true;
+                                } else {
+                                    continue;
+                                }
+                            }
+
+                            /**
+                             * read the split
+                             */
+                            RecordReader reader = inputFormat.getRecordReader(inputSplits[i], conf, Reporter.NULL);
+                            Object key = reader.createKey();
+                            Object value = reader.createValue();
+                            while (reader.next(key, value) == true) {
+                                parser.parse(key, value, writer);
+                            }
+                        }
+                    }
+                    parser.close(writer);
+                    writer.close();
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                } finally {
+                    Thread.currentThread().setContextClassLoader(ctxCL);
+                }
+            }
+        };
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
new file mode 100644
index 0000000..3ce6b2a
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright 2009-2012 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.hdfs.dataflow;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobConf;
+
+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.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriter;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
+
+/**
+ * The HDFS file write operator using the Hadoop old API.
+ * To use this operator, a user need to provide an ITupleWriterFactory.
+ */
+@SuppressWarnings("deprecation")
+public class HDFSWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private ConfFactory confFactory;
+    private ITupleWriterFactory tupleWriterFactory;
+
+    /**
+     * The constructor of HDFSWriteOperatorDescriptor.
+     * 
+     * @param spec
+     *            the JobSpecification object
+     * @param conf
+     *            the Hadoop JobConf which contains the output path
+     * @param tupleWriterFactory
+     *            the ITupleWriterFactory implementation object
+     * @throws HyracksException
+     */
+    public HDFSWriteOperatorDescriptor(JobSpecification spec, JobConf conf, ITupleWriterFactory tupleWriterFactory)
+            throws HyracksException {
+        super(spec, 1, 0);
+        this.confFactory = new ConfFactory(conf);
+        this.tupleWriterFactory = tupleWriterFactory;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+            throws HyracksDataException {
+
+        return new AbstractUnaryInputSinkOperatorNodePushable() {
+
+            private FSDataOutputStream dos;
+            private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
+            private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRd);
+            private FrameTupleReference tuple = new FrameTupleReference();
+            private ITupleWriter tupleWriter;
+            private ClassLoader ctxCL;
+
+            @Override
+            public void open() throws HyracksDataException {
+                ctxCL = Thread.currentThread().getContextClassLoader();
+                Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+                JobConf conf = confFactory.getConf();
+                String outputDirPath = FileOutputFormat.getOutputPath(conf).toString();
+                String fileName = outputDirPath + File.separator + "part-" + partition;
+
+                tupleWriter = tupleWriterFactory.getTupleWriter(ctx);
+                try {
+                    FileSystem dfs = FileSystem.get(conf);
+                    dos = dfs.create(new Path(fileName), true);
+                    tupleWriter.open(dos);
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                accessor.reset(buffer);
+                int tupleCount = accessor.getTupleCount();
+                for (int i = 0; i < tupleCount; i++) {
+                    tuple.reset(accessor, i);
+                    tupleWriter.write(dos, tuple);
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                try {
+                    tupleWriter.close(dos);
+                    dos.close();
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                } finally {
+                    Thread.currentThread().setContextClassLoader(ctxCL);
+                }
+            }
+
+        };
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java
new file mode 100644
index 0000000..147e872
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2012 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.hdfs.dataflow;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings({ "deprecation", "rawtypes" })
+public class InputSplitsFactory implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+    private byte[] splitBytes;
+    private String splitClassName;
+
+    public InputSplitsFactory(InputSplit[] splits) throws HyracksDataException {
+        splitBytes = splitsToBytes(splits);
+        if (splits.length > 0) {
+            splitClassName = splits[0].getClass().getName();
+        } else {
+            splitClassName = FileSplit.class.getName();
+        }
+    }
+
+    public InputSplit[] getSplits() throws HyracksDataException {
+        return bytesToSplits(splitBytes);
+    }
+
+    /**
+     * Convert splits to bytes.
+     * 
+     * @param splits
+     *            input splits
+     * @return bytes which serialize the splits
+     * @throws IOException
+     */
+    private byte[] splitsToBytes(InputSplit[] splits) throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutputStream dos = new DataOutputStream(bos);
+            dos.writeInt(splits.length);
+            for (int i = 0; i < splits.length; i++) {
+                splits[i].write(dos);
+            }
+            dos.close();
+            return bos.toByteArray();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    /**
+     * Covert bytes to splits.
+     * 
+     * @param bytes
+     * @return
+     * @throws HyracksDataException
+     */
+    private InputSplit[] bytesToSplits(byte[] bytes) throws HyracksDataException {
+        try {
+            Class splitClass = Class.forName(splitClassName);
+            Constructor[] constructors = splitClass.getDeclaredConstructors();
+            Constructor defaultConstructor = null;
+            for (Constructor constructor : constructors) {
+                if (constructor.getParameterTypes().length == 0) {
+                    constructor.setAccessible(true);
+                    defaultConstructor = constructor;
+                }
+            }
+            ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
+            DataInputStream dis = new DataInputStream(bis);
+            int size = dis.readInt();
+            InputSplit[] splits = new InputSplit[size];
+            for (int i = 0; i < size; i++) {
+                splits[i] = (InputSplit) defaultConstructor.newInstance();
+                splits[i].readFields(dis);
+            }
+            dis.close();
+            return splits;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryComparatorFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
new file mode 100644
index 0000000..90c5977
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
@@ -0,0 +1,32 @@
+package edu.uci.ics.hyracks.hdfs.lib;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class RawBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static IBinaryComparatorFactory INSTANCE = new RawBinaryComparatorFactory();
+
+    private RawBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int commonLength = Math.min(l1, l2);
+                for (int i = 0; i < commonLength; i++) {
+                    if (b1[s1 + i] != b2[s2 + i]) {
+                        return b1[s1 + i] - b2[s2 + i];
+                    }
+                }
+                int difference = l1 - l2;
+                return difference == 0 ? 0 : (difference > 0 ? 1 : -1);
+            }
+
+        };
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryHashFunctionFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..7895fec
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryHashFunctionFactory.java
@@ -0,0 +1,29 @@
+package edu.uci.ics.hyracks.hdfs.lib;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class RawBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static IBinaryHashFunctionFactory INSTANCE = new RawBinaryHashFunctionFactory();
+
+    private RawBinaryHashFunctionFactory() {
+    }
+
+    @Override
+    public IBinaryHashFunction createBinaryHashFunction() {
+
+        return new IBinaryHashFunction() {
+            @Override
+            public int hash(byte[] bytes, int offset, int length) {
+                int value = 1;
+                int end = offset + length;
+                for (int i = offset; i < end; i++)
+                    value = value * 31 + (int) bytes[i];
+                return value;
+            }
+        };
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
new file mode 100644
index 0000000..9574bb4
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2009-2012 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.hdfs.lib;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.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.hdfs.api.IKeyValueParser;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParserFactory;
+
+public class TextKeyValueParserFactory implements IKeyValueParserFactory<LongWritable, Text> {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IKeyValueParser<LongWritable, Text> createKeyValueParser(final IHyracksTaskContext ctx) {
+
+        final ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
+        final ByteBuffer buffer = ctx.allocateFrame();
+        final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        appender.reset(buffer, true);
+
+        return new IKeyValueParser<LongWritable, Text>() {
+
+            @Override
+            public void open(IFrameWriter writer) {
+
+            }
+
+            @Override
+            public void parse(LongWritable key, Text value, IFrameWriter writer) throws HyracksDataException {
+                tb.reset();
+                tb.addField(value.getBytes(), 0, value.getLength());
+                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                    FrameUtils.flushFrame(buffer, writer);
+                    appender.reset(buffer, true);
+                    if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                        throw new HyracksDataException("tuple cannot be appended into the frame");
+                    }
+                }
+            }
+
+            @Override
+            public void close(IFrameWriter writer) throws HyracksDataException {
+                FrameUtils.flushFrame(buffer, writer);
+            }
+
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextTupleWriterFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextTupleWriterFactory.java
new file mode 100644
index 0000000..0da14e5
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextTupleWriterFactory.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2012 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.hdfs.lib;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriter;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
+
+public class TextTupleWriterFactory implements ITupleWriterFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public ITupleWriter getTupleWriter(IHyracksTaskContext ctx) {
+        return new ITupleWriter() {
+            private byte newLine = "\n".getBytes()[0];
+
+            @Override
+            public void open(DataOutput output) {
+
+            }
+
+            @Override
+            public void write(DataOutput output, ITupleReference tuple) throws HyracksDataException {
+                byte[] data = tuple.getFieldData(0);
+                int start = tuple.getFieldStart(0);
+                int len = tuple.getFieldLength(0);
+                try {
+                    output.write(data, start, len);
+                    output.writeByte(newLine);
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public void close(DataOutput output) {
+
+            }
+
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/IPProximityNcCollectionBuilder.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/IPProximityNcCollectionBuilder.java
new file mode 100644
index 0000000..320b48b
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/IPProximityNcCollectionBuilder.java
@@ -0,0 +1,121 @@
+package edu.uci.ics.hyracks.hdfs.scheduler;
+
+import java.net.InetAddress;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.hdfs.api.INcCollection;
+import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
+
+@SuppressWarnings("deprecation")
+public class IPProximityNcCollectionBuilder implements INcCollectionBuilder {
+
+    @Override
+    public INcCollection build(Map<String, NodeControllerInfo> ncNameToNcInfos,
+            final Map<String, List<String>> ipToNcMapping, final Map<String, Integer> ncNameToIndex, String[] NCs,
+            final int[] workloads, final int slotLimit) {
+        final TreeMap<BytesWritable, IntWritable> availableIpsToSlots = new TreeMap<BytesWritable, IntWritable>();
+        for (int i = 0; i < workloads.length; i++) {
+            if (workloads[i] < slotLimit) {
+                BytesWritable ip = new BytesWritable(ncNameToNcInfos.get(NCs[i]).getNetworkAddress().getIpAddress());
+                IntWritable availableSlot = availableIpsToSlots.get(ip);
+                if (availableSlot == null) {
+                    availableSlot = new IntWritable(slotLimit - workloads[i]);
+                    availableIpsToSlots.put(ip, availableSlot);
+                } else {
+                    availableSlot.set(slotLimit - workloads[i] + availableSlot.get());
+                }
+            }
+        }
+        return new INcCollection() {
+
+            @Override
+            public String findNearestAvailableSlot(InputSplit split) {
+                try {
+                    String[] locs = split.getLocations();
+                    int minDistance = Integer.MAX_VALUE;
+                    BytesWritable currentCandidateIp = null;
+                    if (locs == null || locs.length > 0) {
+                        for (int j = 0; j < locs.length; j++) {
+                            /**
+                             * get all the IP addresses from the name
+                             */
+                            InetAddress[] allIps = InetAddress.getAllByName(locs[j]);
+                            for (InetAddress ip : allIps) {
+                                BytesWritable splitIp = new BytesWritable(ip.getAddress());
+                                /**
+                                 * if the node controller exists
+                                 */
+                                BytesWritable candidateNcIp = availableIpsToSlots.floorKey(splitIp);
+                                if (candidateNcIp == null) {
+                                    candidateNcIp = availableIpsToSlots.ceilingKey(splitIp);
+                                }
+                                if (candidateNcIp != null) {
+                                    if (availableIpsToSlots.get(candidateNcIp).get() > 0) {
+                                        byte[] candidateIP = candidateNcIp.getBytes();
+                                        byte[] splitIP = splitIp.getBytes();
+                                        int candidateInt = candidateIP[0] << 24 | (candidateIP[1] & 0xFF) << 16
+                                                | (candidateIP[2] & 0xFF) << 8 | (candidateIP[3] & 0xFF);
+                                        int splitInt = splitIP[0] << 24 | (splitIP[1] & 0xFF) << 16
+                                                | (splitIP[2] & 0xFF) << 8 | (splitIP[3] & 0xFF);
+                                        int distance = Math.abs(candidateInt - splitInt);
+                                        if (minDistance > distance) {
+                                            minDistance = distance;
+                                            currentCandidateIp = candidateNcIp;
+                                        }
+                                    }
+                                }
+                            }
+                        }
+                    } else {
+                        for (Entry<BytesWritable, IntWritable> entry : availableIpsToSlots.entrySet()) {
+                            if (entry.getValue().get() > 0) {
+                                currentCandidateIp = entry.getKey();
+                                break;
+                            }
+                        }
+                    }
+
+                    if (currentCandidateIp != null) {
+                        /**
+                         * Update the entry of the selected IP
+                         */
+                        IntWritable availableSlot = availableIpsToSlots.get(currentCandidateIp);
+                        availableSlot.set(availableSlot.get() - 1);
+                        if (availableSlot.get() == 0) {
+                            availableIpsToSlots.remove(currentCandidateIp);
+                        }
+                        /**
+                         * Update the entry of the selected NC
+                         */
+                        List<String> dataLocations = ipToNcMapping.get(InetAddress.getByAddress(
+                                currentCandidateIp.getBytes()).getHostAddress());
+                        for (String nc : dataLocations) {
+                            int ncIndex = ncNameToIndex.get(nc);
+                            if (workloads[ncIndex] < slotLimit) {
+                                return nc;
+                            }
+                        }
+                    }
+                    /** not scheduled */
+                    return null;
+                } catch (Exception e) {
+                    throw new IllegalStateException(e);
+                }
+            }
+
+            @Override
+            public int numAvailableSlots() {
+                return availableIpsToSlots.size();
+            }
+
+        };
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/RackAwareNcCollectionBuilder.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/RackAwareNcCollectionBuilder.java
new file mode 100644
index 0000000..fa5e7ae
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/RackAwareNcCollectionBuilder.java
@@ -0,0 +1,211 @@
+package edu.uci.ics.hyracks.hdfs.scheduler;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+import java.util.logging.Logger;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.topology.ClusterTopology;
+import edu.uci.ics.hyracks.hdfs.api.INcCollection;
+import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
+
+@SuppressWarnings("deprecation")
+public class RackAwareNcCollectionBuilder implements INcCollectionBuilder {
+    private static final Logger LOGGER = Logger.getLogger(RackAwareNcCollectionBuilder.class.getName());
+    private ClusterTopology topology;
+
+    public RackAwareNcCollectionBuilder(ClusterTopology topology) {
+        this.topology = topology;
+    }
+
+    @Override
+    public INcCollection build(Map<String, NodeControllerInfo> ncNameToNcInfos,
+            final Map<String, List<String>> ipToNcMapping, final Map<String, Integer> ncNameToIndex, String[] NCs,
+            final int[] workloads, final int slotLimit) {
+        try {
+            final Map<List<Integer>, List<String>> pathToNCs = new HashMap<List<Integer>, List<String>>();
+            for (int i = 0; i < NCs.length; i++) {
+                List<Integer> path = new ArrayList<Integer>();
+                String ipAddress = InetAddress.getByAddress(
+                        ncNameToNcInfos.get(NCs[i]).getNetworkAddress().getIpAddress()).getHostAddress();
+                topology.lookupNetworkTerminal(ipAddress, path);
+                if (path.size() <= 0) {
+                    // if the hyracks nc is not in the defined cluster
+                    path.add(Integer.MIN_VALUE);
+                    LOGGER.info(NCs[i] + "'s IP address is not in the cluster toplogy file!");
+                }
+                List<String> ncs = pathToNCs.get(path);
+                if (ncs == null) {
+                    ncs = new ArrayList<String>();
+                    pathToNCs.put(path, ncs);
+                }
+                ncs.add(NCs[i]);
+            }
+
+            final TreeMap<List<Integer>, IntWritable> availableIpsToSlots = new TreeMap<List<Integer>, IntWritable>(
+                    new Comparator<List<Integer>>() {
+
+                        @Override
+                        public int compare(List<Integer> l1, List<Integer> l2) {
+                            int commonLength = Math.min(l1.size(), l2.size());
+                            for (int i = 0; i < commonLength; i++) {
+                                Integer value1 = l1.get(i);
+                                Integer value2 = l2.get(i);
+                                int cmp = value1 > value2 ? 1 : (value1 < value2 ? -1 : 0);
+                                if (cmp != 0) {
+                                    return cmp;
+                                }
+                            }
+                            return l1.size() > l2.size() ? 1 : (l1.size() < l2.size() ? -1 : 0);
+                        }
+
+                    });
+            for (int i = 0; i < workloads.length; i++) {
+                if (workloads[i] < slotLimit) {
+                    List<Integer> path = new ArrayList<Integer>();
+                    String ipAddress = InetAddress.getByAddress(
+                            ncNameToNcInfos.get(NCs[i]).getNetworkAddress().getIpAddress()).getHostAddress();
+                    topology.lookupNetworkTerminal(ipAddress, path);
+                    if (path.size() <= 0) {
+                        // if the hyracks nc is not in the defined cluster
+                        path.add(Integer.MIN_VALUE);
+                    }
+                    IntWritable availableSlot = availableIpsToSlots.get(path);
+                    if (availableSlot == null) {
+                        availableSlot = new IntWritable(slotLimit - workloads[i]);
+                        availableIpsToSlots.put(path, availableSlot);
+                    } else {
+                        availableSlot.set(slotLimit - workloads[i] + availableSlot.get());
+                    }
+                }
+            }
+            return new INcCollection() {
+
+                @Override
+                public String findNearestAvailableSlot(InputSplit split) {
+                    try {
+                        String[] locs = split.getLocations();
+                        int minDistance = Integer.MAX_VALUE;
+                        List<Integer> currentCandidatePath = null;
+                        if (locs == null || locs.length > 0) {
+                            for (int j = 0; j < locs.length; j++) {
+                                /**
+                                 * get all the IP addresses from the name
+                                 */
+                                InetAddress[] allIps = InetAddress.getAllByName(locs[j]);
+                                boolean inTopology = false;
+                                for (InetAddress ip : allIps) {
+                                    List<Integer> splitPath = new ArrayList<Integer>();
+                                    boolean inCluster = topology.lookupNetworkTerminal(ip.getHostAddress(), splitPath);
+                                    if (!inCluster) {
+                                        continue;
+                                    }
+                                    inTopology = true;
+                                    /**
+                                     * if the node controller exists
+                                     */
+                                    List<Integer> candidatePath = availableIpsToSlots.floorKey(splitPath);
+                                    if (candidatePath == null) {
+                                        candidatePath = availableIpsToSlots.ceilingKey(splitPath);
+                                    }
+                                    if (candidatePath != null) {
+                                        if (availableIpsToSlots.get(candidatePath).get() > 0) {
+                                            int distance = distance(splitPath, candidatePath);
+                                            if (minDistance > distance) {
+                                                minDistance = distance;
+                                                currentCandidatePath = candidatePath;
+                                            }
+                                        }
+
+                                    }
+                                }
+
+                                if (!inTopology) {
+                                    LOGGER.info(locs[j] + "'s IP address is not in the cluster toplogy file!");
+                                    /**
+                                     * if the machine is not in the toplogy file
+                                     */
+                                    List<Integer> candidatePath = null;
+                                    for (Entry<List<Integer>, IntWritable> entry : availableIpsToSlots.entrySet()) {
+                                        if (entry.getValue().get() > 0) {
+                                            candidatePath = entry.getKey();
+                                            break;
+                                        }
+                                    }
+                                    /** the split path is empty */
+                                    if (candidatePath != null) {
+                                        if (availableIpsToSlots.get(candidatePath).get() > 0) {
+                                            currentCandidatePath = candidatePath;
+                                        }
+                                    }
+                                }
+                            }
+                        } else {
+                            for (Entry<List<Integer>, IntWritable> entry : availableIpsToSlots.entrySet()) {
+                                if (entry.getValue().get() > 0) {
+                                    currentCandidatePath = entry.getKey();
+                                    break;
+                                }
+                            }
+                        }
+
+                        if (currentCandidatePath != null && currentCandidatePath.size() > 0) {
+                            /**
+                             * Update the entry of the selected IP
+                             */
+                            IntWritable availableSlot = availableIpsToSlots.get(currentCandidatePath);
+                            availableSlot.set(availableSlot.get() - 1);
+                            if (availableSlot.get() == 0) {
+                                availableIpsToSlots.remove(currentCandidatePath);
+                            }
+                            /**
+                             * Update the entry of the selected NC
+                             */
+                            List<String> candidateNcs = pathToNCs.get(currentCandidatePath);
+                            for (String candidate : candidateNcs) {
+                                int ncIndex = ncNameToIndex.get(candidate);
+                                if (workloads[ncIndex] < slotLimit) {
+                                    return candidate;
+                                }
+                            }
+                        }
+                        /** not scheduled */
+                        return null;
+                    } catch (Exception e) {
+                        throw new IllegalStateException(e);
+                    }
+                }
+
+                @Override
+                public int numAvailableSlots() {
+                    return availableIpsToSlots.size();
+                }
+
+                private int distance(List<Integer> splitPath, List<Integer> candidatePath) {
+                    int commonLength = Math.min(splitPath.size(), candidatePath.size());
+                    int distance = 0;
+                    for (int i = 0; i < commonLength; i++) {
+                        distance = distance * 100 + Math.abs(splitPath.get(i) - candidatePath.get(i));
+                    }
+                    List<Integer> restElements = splitPath.size() > candidatePath.size() ? splitPath : candidatePath;
+                    for (int i = commonLength; i < restElements.size(); i++) {
+                        distance = distance * 100 + Math.abs(restElements.get(i));
+                    }
+                    return distance;
+                }
+            };
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java
new file mode 100644
index 0000000..3f7997b
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java
@@ -0,0 +1,404 @@
+/*
+ * Copyright 2009-2012 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.hdfs.scheduler;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Random;
+import java.util.logging.Logger;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.topology.ClusterTopology;
+import edu.uci.ics.hyracks.hdfs.api.INcCollection;
+import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
+
+/**
+ * The scheduler conduct data-local scheduling for data reading on HDFS. This
+ * class works for Hadoop old API.
+ */
+@SuppressWarnings("deprecation")
+public class Scheduler {
+    private static final Logger LOGGER = Logger.getLogger(Scheduler.class.getName());
+
+    /** a list of NCs */
+    private String[] NCs;
+
+    /** a map from ip to NCs */
+    private Map<String, List<String>> ipToNcMapping = new HashMap<String, List<String>>();
+
+    /** a map from the NC name to the index */
+    private Map<String, Integer> ncNameToIndex = new HashMap<String, Integer>();
+
+    /** a map from NC name to the NodeControllerInfo */
+    private Map<String, NodeControllerInfo> ncNameToNcInfos;
+
+    /**
+     * the nc collection builder
+     */
+    private INcCollectionBuilder ncCollectionBuilder;
+
+    /**
+     * The constructor of the scheduler.
+     * 
+     * @param ncNameToNcInfos
+     * @throws HyracksException
+     */
+    public Scheduler(String ipAddress, int port) throws HyracksException {
+        try {
+            IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
+            this.ncNameToNcInfos = hcc.getNodeControllerInfos();
+            ClusterTopology topology = hcc.getClusterTopology();
+            this.ncCollectionBuilder = topology == null ? new IPProximityNcCollectionBuilder()
+                    : new RackAwareNcCollectionBuilder(topology);
+            loadIPAddressToNCMap(ncNameToNcInfos);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    /**
+     * The constructor of the scheduler.
+     * 
+     * @param ncNameToNcInfos
+     * @throws HyracksException
+     */
+    public Scheduler(String ipAddress, int port, INcCollectionBuilder ncCollectionBuilder) throws HyracksException {
+        try {
+            IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
+            this.ncNameToNcInfos = hcc.getNodeControllerInfos();
+            this.ncCollectionBuilder = ncCollectionBuilder;
+            loadIPAddressToNCMap(ncNameToNcInfos);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    /**
+     * The constructor of the scheduler.
+     * 
+     * @param ncNameToNcInfos
+     *            the mapping from nc names to nc infos
+     * @throws HyracksException
+     */
+    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
+        this.ncNameToNcInfos = ncNameToNcInfos;
+        this.ncCollectionBuilder = new IPProximityNcCollectionBuilder();
+        loadIPAddressToNCMap(ncNameToNcInfos);
+    }
+
+    /**
+     * The constructor of the scheduler.
+     * 
+     * @param ncNameToNcInfos
+     *            the mapping from nc names to nc infos
+     * @param topology
+     *            the hyracks cluster toplogy
+     * @throws HyracksException
+     */
+    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, ClusterTopology topology) throws HyracksException {
+        this(ncNameToNcInfos);
+        this.ncCollectionBuilder = topology == null ? new IPProximityNcCollectionBuilder()
+                : new RackAwareNcCollectionBuilder(topology);
+    }
+
+    /**
+     * The constructor of the scheduler.
+     * 
+     * @param ncNameToNcInfos
+     *            the mapping from nc names to nc infos
+     * @throws HyracksException
+     */
+    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, INcCollectionBuilder ncCollectionBuilder)
+            throws HyracksException {
+        this.ncNameToNcInfos = ncNameToNcInfos;
+        this.ncCollectionBuilder = ncCollectionBuilder;
+        loadIPAddressToNCMap(ncNameToNcInfos);
+    }
+
+    /**
+     * Set location constraints for a file scan operator with a list of file
+     * splits. It guarantees the maximum slots a machine can is at most one more
+     * than the minimum slots a machine can get.
+     * 
+     * @throws HyracksDataException
+     */
+    public String[] getLocationConstraints(InputSplit[] splits) throws HyracksException {
+        if (splits == null) {
+            /** deal the case when the splits array is null */
+            return new String[] {};
+        }
+        int[] workloads = new int[NCs.length];
+        Arrays.fill(workloads, 0);
+        String[] locations = new String[splits.length];
+        Map<String, IntWritable> locationToNumOfSplits = new HashMap<String, IntWritable>();
+        /**
+         * upper bound number of slots that a machine can get
+         */
+        int upperBoundSlots = splits.length % workloads.length == 0 ? (splits.length / workloads.length)
+                : (splits.length / workloads.length + 1);
+        /**
+         * lower bound number of slots that a machine can get
+         */
+        int lowerBoundSlots = splits.length % workloads.length == 0 ? upperBoundSlots : upperBoundSlots - 1;
+
+        try {
+            Random random = new Random(System.currentTimeMillis());
+            boolean scheduled[] = new boolean[splits.length];
+            Arrays.fill(scheduled, false);
+            /**
+             * scan the splits and build the popularity map
+             * give the machines with less local splits more scheduling priority
+             */
+            buildPopularityMap(splits, locationToNumOfSplits);
+            /**
+             * push data-local lower-bounds slots to each machine
+             */
+            scheduleLocalSlots(splits, workloads, locations, lowerBoundSlots, random, scheduled, locationToNumOfSplits);
+            /**
+             * push data-local upper-bounds slots to each machine
+             */
+            scheduleLocalSlots(splits, workloads, locations, upperBoundSlots, random, scheduled, locationToNumOfSplits);
+
+            int dataLocalCount = 0;
+            for (int i = 0; i < scheduled.length; i++) {
+                if (scheduled[i] == true) {
+                    dataLocalCount++;
+                }
+            }
+            LOGGER.info("Data local rate: "
+                    + (scheduled.length == 0 ? 0.0 : ((float) dataLocalCount / (float) (scheduled.length))));
+            /**
+             * push non-data-local lower-bounds slots to each machine
+             */
+            scheduleNonLocalSlots(splits, workloads, locations, lowerBoundSlots, scheduled);
+            /**
+             * push non-data-local upper-bounds slots to each machine
+             */
+            scheduleNonLocalSlots(splits, workloads, locations, upperBoundSlots, scheduled);
+            return locations;
+        } catch (IOException e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    /**
+     * Schedule non-local slots to each machine
+     * 
+     * @param splits
+     *            The HDFS file splits.
+     * @param workloads
+     *            The current capacity of each machine.
+     * @param locations
+     *            The result schedule.
+     * @param slotLimit
+     *            The maximum slots of each machine.
+     * @param scheduled
+     *            Indicate which slot is scheduled.
+     */
+    private void scheduleNonLocalSlots(InputSplit[] splits, int[] workloads, String[] locations, int slotLimit,
+            boolean[] scheduled) throws IOException, UnknownHostException {
+        /**
+         * build the map from available ips to the number of available slots
+         */
+        INcCollection ncCollection = this.ncCollectionBuilder.build(ncNameToNcInfos, ipToNcMapping, ncNameToIndex, NCs,
+                workloads, slotLimit);
+        if (ncCollection.numAvailableSlots() == 0) {
+            return;
+        }
+        /**
+         * schedule no-local file reads
+         */
+        for (int i = 0; i < splits.length; i++) {
+            /** if there is no data-local NC choice, choose a random one */
+            if (!scheduled[i]) {
+                InputSplit split = splits[i];
+                String selectedNcName = ncCollection.findNearestAvailableSlot(split);
+                if (selectedNcName != null) {
+                    int ncIndex = ncNameToIndex.get(selectedNcName);
+                    workloads[ncIndex]++;
+                    scheduled[i] = true;
+                    locations[i] = selectedNcName;
+                }
+            }
+        }
+    }
+
+    /**
+     * Schedule data-local slots to each machine.
+     * 
+     * @param splits
+     *            The HDFS file splits.
+     * @param workloads
+     *            The current capacity of each machine.
+     * @param locations
+     *            The result schedule.
+     * @param slots
+     *            The maximum slots of each machine.
+     * @param random
+     *            The random generator.
+     * @param scheduled
+     *            Indicate which slot is scheduled.
+     * @throws IOException
+     * @throws UnknownHostException
+     */
+    private void scheduleLocalSlots(InputSplit[] splits, int[] workloads, String[] locations, int slots, Random random,
+            boolean[] scheduled, final Map<String, IntWritable> locationToNumSplits) throws IOException,
+            UnknownHostException {
+        /** scheduling candidates will be ordered inversely according to their popularity */
+        PriorityQueue<String> scheduleCadndiates = new PriorityQueue<String>(3, new Comparator<String>() {
+
+            @Override
+            public int compare(String s1, String s2) {
+                return locationToNumSplits.get(s1).compareTo(locationToNumSplits.get(s2));
+            }
+
+        });
+        for (int i = 0; i < splits.length; i++) {
+            if (scheduled[i]) {
+                continue;
+            }
+            /**
+             * get the location of all the splits
+             */
+            String[] locs = splits[i].getLocations();
+            if (locs.length > 0) {
+                scheduleCadndiates.clear();
+                for (int j = 0; j < locs.length; j++) {
+                    scheduleCadndiates.add(locs[j]);
+                }
+
+                for (String candidate : scheduleCadndiates) {
+                    /**
+                     * get all the IP addresses from the name
+                     */
+                    InetAddress[] allIps = InetAddress.getAllByName(candidate);
+                    /**
+                     * iterate overa all ips
+                     */
+                    for (InetAddress ip : allIps) {
+                        /**
+                         * if the node controller exists
+                         */
+                        if (ipToNcMapping.get(ip.getHostAddress()) != null) {
+                            /**
+                             * set the ncs
+                             */
+                            List<String> dataLocations = ipToNcMapping.get(ip.getHostAddress());
+                            int arrayPos = random.nextInt(dataLocations.size());
+                            String nc = dataLocations.get(arrayPos);
+                            int pos = ncNameToIndex.get(nc);
+                            /**
+                             * check if the node is already full
+                             */
+                            if (workloads[pos] < slots) {
+                                locations[i] = nc;
+                                workloads[pos]++;
+                                scheduled[i] = true;
+                                break;
+                            }
+                        }
+                    }
+                    /**
+                     * break the loop for data-locations if the schedule has
+                     * already been found
+                     */
+                    if (scheduled[i] == true) {
+                        break;
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * Scan the splits once and build a popularity map
+     * 
+     * @param splits
+     *            the split array
+     * @param locationToNumOfSplits
+     *            the map to be built
+     * @throws IOException
+     */
+    private void buildPopularityMap(InputSplit[] splits, Map<String, IntWritable> locationToNumOfSplits)
+            throws IOException {
+        for (InputSplit split : splits) {
+            String[] locations = split.getLocations();
+            for (String loc : locations) {
+                IntWritable locCount = locationToNumOfSplits.get(loc);
+                if (locCount == null) {
+                    locCount = new IntWritable(0);
+                    locationToNumOfSplits.put(loc, locCount);
+                }
+                locCount.set(locCount.get() + 1);
+            }
+        }
+    }
+
+    /**
+     * Load the IP-address-to-NC map from the NCNameToNCInfoMap
+     * 
+     * @param ncNameToNcInfos
+     * @throws HyracksException
+     */
+    private void loadIPAddressToNCMap(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
+        try {
+            NCs = new String[ncNameToNcInfos.size()];
+            ipToNcMapping.clear();
+            ncNameToIndex.clear();
+            int i = 0;
+
+            /**
+             * build the IP address to NC map
+             */
+            for (Map.Entry<String, NodeControllerInfo> entry : ncNameToNcInfos.entrySet()) {
+                String ipAddr = InetAddress.getByAddress(entry.getValue().getNetworkAddress().getIpAddress())
+                        .getHostAddress();
+                List<String> matchedNCs = ipToNcMapping.get(ipAddr);
+                if (matchedNCs == null) {
+                    matchedNCs = new ArrayList<String>();
+                    ipToNcMapping.put(ipAddr, matchedNCs);
+                }
+                matchedNCs.add(entry.getKey());
+                NCs[i] = entry.getKey();
+                i++;
+            }
+
+            /**
+             * set up the NC name to index mapping
+             */
+            for (i = 0; i < NCs.length; i++) {
+                ncNameToIndex.put(NCs[i], i);
+            }
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/ConfFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/ConfFactory.java
new file mode 100644
index 0000000..d843d27
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/ConfFactory.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hyracks.hdfs2.dataflow;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.Serializable;
+
+import org.apache.hadoop.mapreduce.Job;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ConfFactory implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private byte[] confBytes;
+
+    public ConfFactory(Job conf) throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutputStream dos = new DataOutputStream(bos);
+            conf.getConfiguration().write(dos);
+            confBytes = bos.toByteArray();
+            dos.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public Job getConf() throws HyracksDataException {
+        try {
+            Job conf = new Job();
+            DataInputStream dis = new DataInputStream(new ByteArrayInputStream(confBytes));
+            conf.getConfiguration().readFields(dis);
+            dis.close();
+            return conf;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/FileSplitsFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/FileSplitsFactory.java
new file mode 100644
index 0000000..14dc70c
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/FileSplitsFactory.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2012 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.hdfs2.dataflow;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings("rawtypes")
+public class FileSplitsFactory implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+    private byte[] splitBytes;
+    private String splitClassName;
+
+    public FileSplitsFactory(List<FileSplit> splits) throws HyracksDataException {
+        splitBytes = splitsToBytes(splits);
+        if (splits.size() > 0) {
+            splitClassName = splits.get(0).getClass().getName();
+        }
+    }
+
+    public List<FileSplit> getSplits() throws HyracksDataException {
+        return bytesToSplits(splitBytes);
+    }
+
+    /**
+     * Convert splits to bytes.
+     * 
+     * @param splits
+     *            input splits
+     * @return bytes which serialize the splits
+     * @throws IOException
+     */
+    private byte[] splitsToBytes(List<FileSplit> splits) throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutputStream dos = new DataOutputStream(bos);
+            dos.writeInt(splits.size());
+            int size = splits.size();
+            for (int i = 0; i < size; i++) {
+                splits.get(i).write(dos);
+            }
+            dos.close();
+            return bos.toByteArray();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    /**
+     * Covert bytes to splits.
+     * 
+     * @param bytes
+     * @return
+     * @throws HyracksDataException
+     */
+    private List<FileSplit> bytesToSplits(byte[] bytes) throws HyracksDataException {
+        try {
+            Class splitClass = Class.forName(splitClassName);
+            Constructor[] constructors = splitClass.getDeclaredConstructors();
+            Constructor defaultConstructor = null;
+            for (Constructor constructor : constructors) {
+                if (constructor.getParameterTypes().length == 0) {
+                    constructor.setAccessible(true);
+                    defaultConstructor = constructor;
+                }
+            }
+            ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
+            DataInputStream dis = new DataInputStream(bis);
+            int size = dis.readInt();
+            List<FileSplit> splits = new ArrayList<FileSplit>();
+            for (int i = 0; i < size; i++) {
+                splits.add((FileSplit) defaultConstructor.newInstance());
+                splits.get(i).readFields(dis);
+            }
+            dis.close();
+            return splits;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java
new file mode 100644
index 0000000..9e9abdf
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java
@@ -0,0 +1,160 @@
+/*
+ * Copyright 2009-2012 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.hdfs2.dataflow;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.util.ReflectionUtils;
+
+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.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.ContextFactory;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParser;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParserFactory;
+
+/**
+ * The HDFS file read operator using the Hadoop new API. To use this operator, a
+ * user need to provide an IKeyValueParserFactory implementation which convert
+ * key-value pairs into tuples.
+ */
+@SuppressWarnings("rawtypes")
+public class HDFSReadOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final ConfFactory confFactory;
+    private final FileSplitsFactory splitsFactory;
+    private final String[] scheduledLocations;
+    private final IKeyValueParserFactory tupleParserFactory;
+    private final boolean[] executed;
+
+    /**
+     * The constructor of HDFSReadOperatorDescriptor.
+     * 
+     * @param spec
+     *            the JobSpecification object
+     * @param rd
+     *            the output record descriptor
+     * @param conf
+     *            the Hadoop JobConf object, which contains the input format and
+     *            the input paths
+     * @param splits
+     *            the array of FileSplits (HDFS chunks).
+     * @param scheduledLocations
+     *            the node controller names to scan the FileSplits, which is an
+     *            one-to-one mapping. The String array is obtained from the
+     *            edu.cui
+     *            .ics.hyracks.hdfs.scheduler.Scheduler.getLocationConstraints
+     *            (InputSplits[]).
+     * @param tupleParserFactory
+     *            the ITupleParserFactory implementation instance.
+     * @throws HyracksException
+     */
+    public HDFSReadOperatorDescriptor(JobSpecification spec, RecordDescriptor rd, Job conf, List<InputSplit> splits,
+            String[] scheduledLocations, IKeyValueParserFactory tupleParserFactory) throws HyracksException {
+        super(spec, 0, 1);
+        try {
+            List<FileSplit> fileSplits = new ArrayList<FileSplit>();
+            for (int i = 0; i < splits.size(); i++) {
+                fileSplits.add((FileSplit) splits.get(i));
+            }
+            this.splitsFactory = new FileSplitsFactory(fileSplits);
+            this.confFactory = new ConfFactory(conf);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+        this.scheduledLocations = scheduledLocations;
+        this.executed = new boolean[scheduledLocations.length];
+        Arrays.fill(executed, false);
+        this.tupleParserFactory = tupleParserFactory;
+        this.recordDescriptors[0] = rd;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+            throws HyracksDataException {
+        final List<FileSplit> inputSplits = splitsFactory.getSplits();
+
+        return new AbstractUnaryOutputSourceOperatorNodePushable() {
+            private String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+            private ContextFactory ctxFactory = new ContextFactory();
+
+            @SuppressWarnings("unchecked")
+            @Override
+            public void initialize() throws HyracksDataException {
+                ClassLoader ctxCL = Thread.currentThread().getContextClassLoader();
+                try {
+                    Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+                    Job job = confFactory.getConf();
+                    IKeyValueParser parser = tupleParserFactory.createKeyValueParser(ctx);
+                    writer.open();
+                    InputFormat inputFormat = ReflectionUtils.newInstance(job.getInputFormatClass(),
+                            job.getConfiguration());
+                    int size = inputSplits.size();
+                    for (int i = 0; i < size; i++) {
+                        /**
+                         * read all the partitions scheduled to the current node
+                         */
+                        if (scheduledLocations[i].equals(nodeName)) {
+                            /**
+                             * pick an unread split to read synchronize among
+                             * simultaneous partitions in the same machine
+                             */
+                            synchronized (executed) {
+                                if (executed[i] == false) {
+                                    executed[i] = true;
+                                } else {
+                                    continue;
+                                }
+                            }
+
+                            /**
+                             * read the split
+                             */
+                            TaskAttemptContext context = ctxFactory.createContext(job.getConfiguration(), i);
+                            RecordReader reader = inputFormat.createRecordReader(inputSplits.get(i), context);
+                            reader.initialize(inputSplits.get(i), context);
+                            while (reader.nextKeyValue() == true) {
+                                parser.parse(reader.getCurrentKey(), reader.getCurrentValue(), writer);
+                            }
+                        }
+                    }
+                    parser.close(writer);
+                    writer.close();
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                } finally {
+                    Thread.currentThread().setContextClassLoader(ctxCL);
+                }
+            }
+        };
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
new file mode 100644
index 0000000..c1c227c
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
@@ -0,0 +1,130 @@
+/*
+ * Copyright 2009-2012 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.hdfs2.dataflow;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+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.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriter;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
+
+/**
+ * The HDFS file write operator using the Hadoop new API. To use this operator,
+ * a user need to provide an ITupleWriterFactory.
+ */
+public class HDFSWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private ConfFactory confFactory;
+    private ITupleWriterFactory tupleWriterFactory;
+
+    /**
+     * The constructor of HDFSWriteOperatorDescriptor.
+     * 
+     * @param spec
+     *            the JobSpecification object
+     * @param conf
+     *            the Hadoop JobConf which contains the output path
+     * @param tupleWriterFactory
+     *            the ITupleWriterFactory implementation object
+     * @throws HyracksException
+     */
+    public HDFSWriteOperatorDescriptor(JobSpecification spec, Job conf, ITupleWriterFactory tupleWriterFactory)
+            throws HyracksException {
+        super(spec, 1, 0);
+        this.confFactory = new ConfFactory(conf);
+        this.tupleWriterFactory = tupleWriterFactory;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+            throws HyracksDataException {
+
+        return new AbstractUnaryInputSinkOperatorNodePushable() {
+
+            private FSDataOutputStream dos;
+            private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
+            private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRd);
+            private FrameTupleReference tuple = new FrameTupleReference();
+            private ITupleWriter tupleWriter;
+            private ClassLoader ctxCL;
+
+            @Override
+            public void open() throws HyracksDataException {
+                ctxCL = Thread.currentThread().getContextClassLoader();
+                Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+                Job conf = confFactory.getConf();
+                String outputPath = FileOutputFormat.getOutputPath(conf).toString();
+                String fileName = outputPath + File.separator + "part-" + partition;
+
+                tupleWriter = tupleWriterFactory.getTupleWriter(ctx);
+                try {
+                    FileSystem dfs = FileSystem.get(conf.getConfiguration());
+                    dos = dfs.create(new Path(fileName), true);
+                    tupleWriter.open(dos);
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                accessor.reset(buffer);
+                int tupleCount = accessor.getTupleCount();
+                for (int i = 0; i < tupleCount; i++) {
+                    tuple.reset(accessor, i);
+                    tupleWriter.write(dos, tuple);
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                try {
+                    tupleWriter.close(dos);
+                    dos.close();
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                } finally {
+                    Thread.currentThread().setContextClassLoader(ctxCL);
+                }
+            }
+
+        };
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java
new file mode 100644
index 0000000..cde187d
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2009-2012 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.hdfs2.scheduler;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.mapreduce.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
+
+/**
+ * The scheduler conduct data-local scheduling for data reading on HDFS.
+ * This class works for Hadoop new API.
+ */
+@SuppressWarnings("deprecation")
+public class Scheduler {
+
+    private edu.uci.ics.hyracks.hdfs.scheduler.Scheduler scheduler;
+
+    /**
+     * The constructor of the scheduler
+     * 
+     * @param ncNameToNcInfos
+     * @throws HyracksException
+     */
+    public Scheduler(String ipAddress, int port) throws HyracksException {
+        scheduler = new edu.uci.ics.hyracks.hdfs.scheduler.Scheduler(ipAddress, port);
+    }
+
+    /**
+     * The constructor of the scheduler.
+     * 
+     * @param ncNameToNcInfos
+     *            the mapping from nc names to nc infos
+     * @throws HyracksException
+     */
+    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
+        scheduler = new edu.uci.ics.hyracks.hdfs.scheduler.Scheduler(ncNameToNcInfos);
+    }
+    
+    /**
+     * The constructor of the scheduler.
+     * 
+     * @param ncNameToNcInfos
+     *            the mapping from nc names to nc infos
+     * @throws HyracksException
+     */
+    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, INcCollectionBuilder builder) throws HyracksException {
+        scheduler = new edu.uci.ics.hyracks.hdfs.scheduler.Scheduler(ncNameToNcInfos, builder);
+    }
+
+    /**
+     * Set location constraints for a file scan operator with a list of file splits
+     * 
+     * @throws HyracksDataException
+     */
+    public String[] getLocationConstraints(List<InputSplit> splits) throws HyracksException {
+        try {
+            org.apache.hadoop.mapred.InputSplit[] inputSplits = new org.apache.hadoop.mapred.InputSplit[splits.size()];
+            for (int i = 0; i < inputSplits.length; i++)
+                inputSplits[i] = new WrappedFileSplit(splits.get(i).getLocations(), splits.get(i).getLength());
+            return scheduler.getLocationConstraints(inputSplits);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/WrappedFileSplit.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/WrappedFileSplit.java
new file mode 100644
index 0000000..1deb469
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/WrappedFileSplit.java
@@ -0,0 +1,51 @@
+package edu.uci.ics.hyracks.hdfs2.scheduler;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.mapred.InputSplit;
+
+/**
+ * The wrapped implementation of InputSplit, for the new API scheduler
+ * to reuse the old API scheduler
+ */
+@SuppressWarnings("deprecation")
+public class WrappedFileSplit implements InputSplit {
+
+    private String[] locations;
+    private long length;
+
+    public WrappedFileSplit(String[] locations, long length) {
+        this.locations = locations;
+        this.length = length;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        int len = input.readInt();
+        locations = new String[len];
+        for (int i = 0; i < len; i++)
+            locations[i] = input.readUTF();
+        length = input.readLong();
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        output.write(locations.length);
+        for (int i = 0; i < locations.length; i++)
+            output.writeUTF(locations[i]);
+        output.writeLong(length);
+    }
+
+    @Override
+    public long getLength() throws IOException {
+        return length;
+    }
+
+    @Override
+    public String[] getLocations() throws IOException {
+        return locations;
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
new file mode 100644
index 0000000..2686077
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
@@ -0,0 +1,204 @@
+/*
+ * Copyright 2009-2012 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.hdfs.dataflow;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.hdfs.lib.RawBinaryComparatorFactory;
+import edu.uci.ics.hyracks.hdfs.lib.RawBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.hdfs.lib.TextKeyValueParserFactory;
+import edu.uci.ics.hyracks.hdfs.lib.TextTupleWriterFactory;
+import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
+import edu.uci.ics.hyracks.hdfs.utils.HyracksUtils;
+import edu.uci.ics.hyracks.hdfs.utils.TestUtils;
+
+/**
+ * Test the edu.uci.ics.hyracks.hdfs.dataflow package,
+ * the operators for the Hadoop old API.
+ */
+@SuppressWarnings({ "deprecation" })
+public class DataflowTest extends TestCase {
+
+    private static final String ACTUAL_RESULT_DIR = "actual";
+    private static final String EXPECTED_RESULT_PATH = "src/test/resources/expected";
+    private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
+
+    private static final String DATA_PATH = "src/test/resources/data/customer.tbl";
+    private static final String HDFS_INPUT_PATH = "/customer/";
+    private static final String HDFS_OUTPUT_PATH = "/customer_result/";
+
+    private static final String HYRACKS_APP_NAME = "DataflowTest";
+    private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
+    private MiniDFSCluster dfsCluster;
+
+    private JobConf conf = new JobConf();
+    private int numberOfNC = 2;
+
+    @Override
+    public void setUp() throws Exception {
+        cleanupStores();
+        HyracksUtils.init();
+        HyracksUtils.createApp(HYRACKS_APP_NAME);
+        FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
+        FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
+        startHDFS();
+    }
+
+    private void cleanupStores() throws IOException {
+        FileUtils.forceMkdir(new File("teststore"));
+        FileUtils.forceMkdir(new File("build"));
+        FileUtils.cleanDirectory(new File("teststore"));
+        FileUtils.cleanDirectory(new File("build"));
+    }
+
+    /**
+     * Start the HDFS cluster and setup the data files
+     * 
+     * @throws IOException
+     */
+    private void startHDFS() throws IOException {
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
+
+        FileSystem lfs = FileSystem.getLocal(new Configuration());
+        lfs.delete(new Path("build"), true);
+        System.setProperty("hadoop.log.dir", "logs");
+        dfsCluster = new MiniDFSCluster(conf, numberOfNC, true, null);
+        FileSystem dfs = FileSystem.get(conf);
+        Path src = new Path(DATA_PATH);
+        Path dest = new Path(HDFS_INPUT_PATH);
+        Path result = new Path(HDFS_OUTPUT_PATH);
+        dfs.mkdirs(dest);
+        dfs.mkdirs(result);
+        dfs.copyFromLocalFile(src, dest);
+
+        DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
+        conf.writeXml(confOutput);
+        confOutput.flush();
+        confOutput.close();
+    }
+
+    /**
+     * Test a job with only HDFS read and writes.
+     * 
+     * @throws Exception
+     */
+    public void testHDFSReadWriteOperators() throws Exception {
+        FileInputFormat.setInputPaths(conf, HDFS_INPUT_PATH);
+        FileOutputFormat.setOutputPath(conf, new Path(HDFS_OUTPUT_PATH));
+        conf.setInputFormat(TextInputFormat.class);
+
+        Scheduler scheduler = new Scheduler(HyracksUtils.CC_HOST, HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
+        InputSplit[] splits = conf.getInputFormat().getSplits(conf, numberOfNC * 4);
+
+        String[] readSchedule = scheduler.getLocationConstraints(splits);
+        JobSpecification jobSpec = new JobSpecification();
+        RecordDescriptor recordDesc = new RecordDescriptor(
+                new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
+
+        String[] locations = new String[] { HyracksUtils.NC1_ID, HyracksUtils.NC1_ID, HyracksUtils.NC2_ID,
+                HyracksUtils.NC2_ID };
+        HDFSReadOperatorDescriptor readOperator = new HDFSReadOperatorDescriptor(jobSpec, recordDesc, conf, splits,
+                readSchedule, new TextKeyValueParserFactory());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, readOperator, locations);
+
+        ExternalSortOperatorDescriptor sortOperator = new ExternalSortOperatorDescriptor(jobSpec, 10, new int[] { 0 },
+                new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }, recordDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, sortOperator, locations);
+
+        HDFSWriteOperatorDescriptor writeOperator = new HDFSWriteOperatorDescriptor(jobSpec, conf,
+                new TextTupleWriterFactory());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, writeOperator, HyracksUtils.NC1_ID);
+
+        jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), readOperator, 0, sortOperator, 0);
+        jobSpec.connect(new MToNPartitioningMergingConnectorDescriptor(jobSpec, new FieldHashPartitionComputerFactory(
+                new int[] { 0 }, new IBinaryHashFunctionFactory[] { RawBinaryHashFunctionFactory.INSTANCE }),
+                new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }), sortOperator,
+                0, writeOperator, 0);
+        jobSpec.addRoot(writeOperator);
+
+        IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
+                HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
+        JobId jobId = client.startJob(HYRACKS_APP_NAME, jobSpec);
+        client.waitForCompletion(jobId);
+
+        Assert.assertEquals(true, checkResults());
+    }
+
+    /**
+     * Check if the results are correct
+     * 
+     * @return true if correct
+     * @throws Exception
+     */
+    private boolean checkResults() throws Exception {
+        FileSystem dfs = FileSystem.get(conf);
+        Path result = new Path(HDFS_OUTPUT_PATH);
+        Path actual = new Path(ACTUAL_RESULT_DIR);
+        dfs.copyToLocalFile(result, actual);
+
+        TestUtils.compareWithResult(new File(EXPECTED_RESULT_PATH + File.separator + "part-0"), new File(
+                ACTUAL_RESULT_DIR + File.separator + "customer_result" + File.separator + "part-0"));
+        return true;
+    }
+
+    /**
+     * cleanup hdfs cluster
+     */
+    private void cleanupHDFS() throws Exception {
+        dfsCluster.shutdown();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        HyracksUtils.destroyApp(HYRACKS_APP_NAME);
+        HyracksUtils.deinit();
+        cleanupHDFS();
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java
new file mode 100644
index 0000000..1f394a9
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java
@@ -0,0 +1,341 @@
+/*
+ * Copyright 2009-2012 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.hdfs.scheduler;
+
+import java.io.FileReader;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.HashMap;
+import java.util.Map;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.client.NodeStatus;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.topology.ClusterTopology;
+import edu.uci.ics.hyracks.api.topology.TopologyDefinitionParser;
+
+@SuppressWarnings("deprecation")
+public class SchedulerTest extends TestCase {
+    private static String TOPOLOGY_PATH = "src/test/resources/topology.xml";
+
+    private ClusterTopology parseTopology() throws IOException, SAXException {
+        FileReader fr = new FileReader(TOPOLOGY_PATH);
+        InputSource in = new InputSource(fr);
+        try {
+            return TopologyDefinitionParser.parse(in);
+        } finally {
+            fr.close();
+        }
+    }
+
+    /**
+     * Test the scheduler for the case when the Hyracks cluster is the HDFS cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerSimple() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
+
+        InputSplit[] fileSplits = new InputSplit[6];
+        fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[1] = new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[2] = new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" });
+        fileSplits[3] = new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" });
+        fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc6", "nc2", "nc3", "nc5" };
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+        ClusterTopology topology = parseTopology();
+        scheduler = new Scheduler(ncNameToNcInfos, topology);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+    /**
+     * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerLargerHDFS() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc7", new NodeControllerInfo("nc7", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.7").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc12", new NodeControllerInfo("nc12", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.12").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
+
+        InputSplit[] fileSplits = new InputSplit[12];
+        fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[1] = new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[2] = new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" });
+        fileSplits[3] = new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" });
+        fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+        fileSplits[6] = new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[7] = new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[8] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.14", "10.0.0.11", "10.0.0.13" });
+        fileSplits[9] = new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" });
+        fileSplits[10] = new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.7" });
+        fileSplits[11] = new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" });
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc12",
+                "nc7", "nc7", "nc12" };
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+        expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc7", "nc12", "nc7",
+                "nc12" };
+        ClusterTopology topology = parseTopology();
+        scheduler = new Scheduler(ncNameToNcInfos, topology);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+    /**
+     * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerSmallerHDFS() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
+
+        InputSplit[] fileSplits = new InputSplit[12];
+        fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[1] = new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[2] = new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.3" });
+        fileSplits[3] = new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.3" });
+        fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+        fileSplits[6] = new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[7] = new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[8] = new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.1" });
+        fileSplits[9] = new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.2" });
+        fileSplits[10] = new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[11] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc5", "nc6",
+                "nc5", "nc6" };
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+        ClusterTopology topology = parseTopology();
+        scheduler = new Scheduler(ncNameToNcInfos, topology);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+    /**
+     * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerSmallerHDFSOdd() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
+
+        InputSplit[] fileSplits = new InputSplit[13];
+        fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[1] = new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[2] = new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.3" });
+        fileSplits[3] = new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.3" });
+        fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+        fileSplits[6] = new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+        fileSplits[7] = new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[8] = new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.1" });
+        fileSplits[9] = new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.2" });
+        fileSplits[10] = new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+        fileSplits[11] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+        fileSplits[12] = new FileSplit(new Path("part-13"), 0, 0, new String[] { "10.0.0.2", "10.0.0.4", "10.0.0.5" });
+
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc5", "nc1",
+                "nc5", "nc2", "nc4" };
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+        ClusterTopology topology = parseTopology();
+        scheduler = new Scheduler(ncNameToNcInfos, topology);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+    }
+
+    /**
+     * Test boundary cases where splits array is empty or null
+     * 
+     * @throws Exception
+     */
+    public void testSchedulercBoundary() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
+
+        /** test empty file splits */
+        InputSplit[] fileSplits = new InputSplit[0];
+        String[] expectedResults = new String[] {};
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+        ClusterTopology topology = parseTopology();
+        scheduler = new Scheduler(ncNameToNcInfos, topology);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+        fileSplits = null;
+        expectedResults = new String[] {};
+
+        scheduler = new Scheduler(ncNameToNcInfos);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+        scheduler = new Scheduler(ncNameToNcInfos, topology);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
new file mode 100644
index 0000000..8c12518
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2012 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.hdfs.utils;
+
+import java.util.EnumSet;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+
+public class HyracksUtils {
+
+    public static final String NC1_ID = "nc1";
+    public static final String NC2_ID = "nc2";
+
+    public static final int DEFAULT_HYRACKS_CC_PORT = 1099;
+    public static final int TEST_HYRACKS_CC_PORT = 1099;
+    public static final int TEST_HYRACKS_CC_CLIENT_PORT = 2099;
+    public static final String CC_HOST = "localhost";
+
+    public static final int FRAME_SIZE = 65536;
+
+    private static ClusterControllerService cc;
+    private static NodeControllerService nc1;
+    private static NodeControllerService nc2;
+    private static IHyracksClientConnection hcc;
+
+    public static void init() throws Exception {
+        CCConfig ccConfig = new CCConfig();
+        ccConfig.clientNetIpAddress = CC_HOST;
+        ccConfig.clusterNetIpAddress = CC_HOST;
+        ccConfig.clusterNetPort = TEST_HYRACKS_CC_PORT;
+        ccConfig.clientNetPort = TEST_HYRACKS_CC_CLIENT_PORT;
+        ccConfig.defaultMaxJobAttempts = 0;
+        ccConfig.jobHistorySize = 0;
+        ccConfig.profileDumpPeriod = -1;
+
+        // cluster controller
+        cc = new ClusterControllerService(ccConfig);
+        cc.start();
+
+        // two node controllers
+        NCConfig ncConfig1 = new NCConfig();
+        ncConfig1.ccHost = "localhost";
+        ncConfig1.clusterNetIPAddress = "localhost";
+        ncConfig1.ccPort = TEST_HYRACKS_CC_PORT;
+        ncConfig1.dataIPAddress = "127.0.0.1";
+        ncConfig1.datasetIPAddress = "127.0.0.1";
+        ncConfig1.nodeId = NC1_ID;
+        nc1 = new NodeControllerService(ncConfig1);
+        nc1.start();
+
+        NCConfig ncConfig2 = new NCConfig();
+        ncConfig2.ccHost = "localhost";
+        ncConfig2.clusterNetIPAddress = "localhost";
+        ncConfig2.ccPort = TEST_HYRACKS_CC_PORT;
+        ncConfig2.dataIPAddress = "127.0.0.1";
+        ncConfig2.datasetIPAddress = "127.0.0.1";
+        ncConfig2.nodeId = NC2_ID;
+        nc2 = new NodeControllerService(ncConfig2);
+        nc2.start();
+
+        // hyracks connection
+        hcc = new HyracksConnection(CC_HOST, TEST_HYRACKS_CC_CLIENT_PORT);
+    }
+
+    public static void destroyApp(String hyracksAppName) throws Exception {
+        hcc.destroyApplication(hyracksAppName);
+    }
+
+    public static void createApp(String hyracksAppName) throws Exception {
+        hcc.createApplication(hyracksAppName, null);
+    }
+
+    public static void deinit() throws Exception {
+        nc2.stop();
+        nc1.stop();
+        cc.stop();
+    }
+
+    public static void runJob(JobSpecification spec, String appName) throws Exception {
+        spec.setFrameSize(FRAME_SIZE);
+        JobId jobId = hcc.startJob(appName, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+        hcc.waitForCompletion(jobId);
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/TestUtils.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/TestUtils.java
new file mode 100644
index 0000000..3826688
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/TestUtils.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2009-2012 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.hdfs.utils;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+
+public class TestUtils {
+
+    public static void compareWithResult(File expectedFile, File actualFile) throws Exception {
+        BufferedReader readerExpected = new BufferedReader(new FileReader(expectedFile));
+        BufferedReader readerActual = new BufferedReader(new FileReader(actualFile));
+        String lineExpected, lineActual;
+        int num = 1;
+        try {
+            while ((lineExpected = readerExpected.readLine()) != null) {
+                lineActual = readerActual.readLine();
+                // Assert.assertEquals(lineExpected, lineActual);
+                if (lineActual == null) {
+                    throw new Exception("Actual result changed at line " + num + ":\n< " + lineExpected + "\n> ");
+                }
+                if (!equalStrings(lineExpected, lineActual)) {
+                    throw new Exception("Result for changed at line " + num + ":\n< " + lineExpected + "\n> "
+                            + lineActual);
+                }
+                ++num;
+            }
+            lineActual = readerActual.readLine();
+            if (lineActual != null) {
+                throw new Exception("Actual result changed at line " + num + ":\n< \n> " + lineActual);
+            }
+        } finally {
+            readerExpected.close();
+            readerActual.close();
+        }
+    }
+
+    private static boolean equalStrings(String s1, String s2) {
+        String[] rowsOne = s1.split("\n");
+        String[] rowsTwo = s2.split("\n");
+
+        if (rowsOne.length != rowsTwo.length)
+            return false;
+
+        for (int i = 0; i < rowsOne.length; i++) {
+            String row1 = rowsOne[i];
+            String row2 = rowsTwo[i];
+
+            if (row1.equals(row2))
+                continue;
+
+            String[] fields1 = row1.split(",");
+            String[] fields2 = row2.split(",");
+
+            for (int j = 0; j < fields1.length; j++) {
+                if (fields1[j].equals(fields2[j])) {
+                    continue;
+                } else if (fields1[j].indexOf('.') < 0) {
+                    return false;
+                } else {
+                    fields1[j] = fields1[j].split("=")[1];
+                    fields2[j] = fields2[j].split("=")[1];
+                    Double double1 = Double.parseDouble(fields1[j]);
+                    Double double2 = Double.parseDouble(fields2[j]);
+                    float float1 = (float) double1.doubleValue();
+                    float float2 = (float) double2.doubleValue();
+
+                    if (Math.abs(float1 - float2) == 0)
+                        continue;
+                    else {
+                        return false;
+                    }
+                }
+            }
+        }
+        return true;
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
new file mode 100644
index 0000000..9f77979
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
@@ -0,0 +1,211 @@
+/*
+ * Copyright 2009-2012 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.hdfs2.dataflow;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.List;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.hdfs.MiniDFSClusterFactory;
+import edu.uci.ics.hyracks.hdfs.lib.RawBinaryComparatorFactory;
+import edu.uci.ics.hyracks.hdfs.lib.RawBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.hdfs.lib.TextKeyValueParserFactory;
+import edu.uci.ics.hyracks.hdfs.lib.TextTupleWriterFactory;
+import edu.uci.ics.hyracks.hdfs.utils.HyracksUtils;
+import edu.uci.ics.hyracks.hdfs.utils.TestUtils;
+import edu.uci.ics.hyracks.hdfs2.scheduler.Scheduler;
+
+/**
+ * Test the edu.uci.ics.hyracks.hdfs2.dataflow package,
+ * the operators for the Hadoop new API.
+ */
+public class DataflowTest extends TestCase {
+
+    private static final String ACTUAL_RESULT_DIR = "actual";
+    private static final String EXPECTED_RESULT_PATH = "src/test/resources/expected";
+    private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
+
+    private static final String DATA_PATH = "src/test/resources/data/customer.tbl";
+    private static final String HDFS_INPUT_PATH = "/customer/";
+    private static final String HDFS_OUTPUT_PATH = "/customer_result/";
+
+    private static final String HYRACKS_APP_NAME = "DataflowTest";
+    private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
+    private MiniDFSCluster dfsCluster;
+    private MiniDFSClusterFactory dfsClusterFactory = new MiniDFSClusterFactory();
+
+    private Job conf;
+    private int numberOfNC = 2;
+
+    @Override
+    public void setUp() throws Exception {
+        conf = new Job();
+        cleanupStores();
+        HyracksUtils.init();
+        HyracksUtils.createApp(HYRACKS_APP_NAME);
+        FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
+        FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
+        startHDFS();
+    }
+
+    private void cleanupStores() throws IOException {
+        FileUtils.forceMkdir(new File("teststore"));
+        FileUtils.forceMkdir(new File("build"));
+        FileUtils.cleanDirectory(new File("teststore"));
+        FileUtils.cleanDirectory(new File("build"));
+    }
+
+    /**
+     * Start the HDFS cluster and setup the data files
+     * 
+     * @throws IOException
+     */
+    private void startHDFS() throws IOException {
+        conf.getConfiguration().addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
+        conf.getConfiguration().addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
+        conf.getConfiguration().addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
+
+        FileSystem lfs = FileSystem.getLocal(new Configuration());
+        lfs.delete(new Path("build"), true);
+        System.setProperty("hadoop.log.dir", "logs");
+        dfsCluster = dfsClusterFactory.getMiniDFSCluster(conf.getConfiguration(), numberOfNC);
+        FileSystem dfs = FileSystem.get(conf.getConfiguration());
+        Path src = new Path(DATA_PATH);
+        Path dest = new Path(HDFS_INPUT_PATH);
+        Path result = new Path(HDFS_OUTPUT_PATH);
+        dfs.mkdirs(dest);
+        dfs.mkdirs(result);
+        dfs.copyFromLocalFile(src, dest);
+
+        DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
+        conf.getConfiguration().writeXml(confOutput);
+        confOutput.flush();
+        confOutput.close();
+    }
+
+    /**
+     * Test a job with only HDFS read and writes.
+     * 
+     * @throws Exception
+     */
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    public void testHDFSReadWriteOperators() throws Exception {
+        FileInputFormat.setInputPaths(conf, HDFS_INPUT_PATH);
+        FileOutputFormat.setOutputPath(conf, new Path(HDFS_OUTPUT_PATH));
+        conf.setInputFormatClass(TextInputFormat.class);
+
+        Scheduler scheduler = new Scheduler(HyracksUtils.CC_HOST, HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
+        InputFormat inputFormat = ReflectionUtils.newInstance(conf.getInputFormatClass(), conf.getConfiguration());
+        List<InputSplit> splits = inputFormat.getSplits(conf);
+
+        String[] readSchedule = scheduler.getLocationConstraints(splits);
+        JobSpecification jobSpec = new JobSpecification();
+        RecordDescriptor recordDesc = new RecordDescriptor(
+                new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
+
+        String[] locations = new String[] { HyracksUtils.NC1_ID, HyracksUtils.NC1_ID, HyracksUtils.NC2_ID,
+                HyracksUtils.NC2_ID };
+        HDFSReadOperatorDescriptor readOperator = new HDFSReadOperatorDescriptor(jobSpec, recordDesc, conf, splits,
+                readSchedule, new TextKeyValueParserFactory());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, readOperator, locations);
+
+        ExternalSortOperatorDescriptor sortOperator = new ExternalSortOperatorDescriptor(jobSpec, 10, new int[] { 0 },
+                new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }, recordDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, sortOperator, locations);
+
+        HDFSWriteOperatorDescriptor writeOperator = new HDFSWriteOperatorDescriptor(jobSpec, conf,
+                new TextTupleWriterFactory());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, writeOperator, HyracksUtils.NC1_ID);
+
+        jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), readOperator, 0, sortOperator, 0);
+        jobSpec.connect(new MToNPartitioningMergingConnectorDescriptor(jobSpec, new FieldHashPartitionComputerFactory(
+                new int[] { 0 }, new IBinaryHashFunctionFactory[] { RawBinaryHashFunctionFactory.INSTANCE }),
+                new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }), sortOperator,
+                0, writeOperator, 0);
+        jobSpec.addRoot(writeOperator);
+
+        IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
+                HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
+        JobId jobId = client.startJob(HYRACKS_APP_NAME, jobSpec);
+        client.waitForCompletion(jobId);
+
+        Assert.assertEquals(true, checkResults());
+    }
+
+    /**
+     * Check if the results are correct
+     * 
+     * @return true if correct
+     * @throws Exception
+     */
+    private boolean checkResults() throws Exception {
+        FileSystem dfs = FileSystem.get(conf.getConfiguration());
+        Path result = new Path(HDFS_OUTPUT_PATH);
+        Path actual = new Path(ACTUAL_RESULT_DIR);
+        dfs.copyToLocalFile(result, actual);
+
+        TestUtils.compareWithResult(new File(EXPECTED_RESULT_PATH + File.separator + "part-0"), new File(
+                ACTUAL_RESULT_DIR + File.separator + "customer_result" + File.separator + "part-0"));
+        return true;
+    }
+
+    /**
+     * cleanup hdfs cluster
+     */
+    private void cleanupHDFS() throws Exception {
+        dfsCluster.shutdown();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        HyracksUtils.destroyApp(HYRACKS_APP_NAME);
+        HyracksUtils.deinit();
+        cleanupHDFS();
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java
new file mode 100644
index 0000000..442aeae0
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java
@@ -0,0 +1,238 @@
+/*
+ * Copyright 2009-2012 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.hdfs2.scheduler;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.client.NodeStatus;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+
+/**
+ * Test case for the new HDFS API scheduler
+ */
+public class SchedulerTest extends TestCase {
+
+    /**
+     * Test the scheduler for the case when the Hyracks cluster is the HDFS cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerSimple() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
+
+        List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+        fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" }));
+        fileSplits.add(new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" }));
+        fileSplits.add(new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc6", "nc2", "nc3", "nc5" };
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+    /**
+     * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerLargerHDFS() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
+
+        List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+        fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" }));
+        fileSplits.add(new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" }));
+        fileSplits.add(new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" }));
+        fileSplits.add(new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" }));
+        fileSplits.add(new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.7" }));
+        fileSplits.add(new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc6", "nc1", "nc4", "nc2", "nc2", "nc3", "nc6", "nc5",
+                "nc3", "nc5" };
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+    /**
+     * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerSmallerHDFS() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
+
+        List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+        fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.1" }));
+        fileSplits.add(new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.2" }));
+        fileSplits.add(new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc5", "nc6",
+                "nc5", "nc6" };
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+    /**
+     * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+     * 
+     * @throws Exception
+     */
+    public void testSchedulerSmallerHDFSOdd() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
+
+        List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+        fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+        fileSplits.add(new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.1" }));
+        fileSplits.add(new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.2" }));
+        fileSplits.add(new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+        fileSplits.add(new FileSplit(new Path("part-13"), 0, 0, new String[] { "10.0.0.2", "10.0.0.4", "10.0.0.5" }));
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc5", "nc1",
+                "nc5", "nc2", "nc4" };
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/data/customer.tbl b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/data/customer.tbl
new file mode 100644
index 0000000..5d39c80
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/data/customer.tbl
@@ -0,0 +1,150 @@
+1|Customer#000000001|IVhzIApeRb ot,c,E|15|25-989-741-2988|711.56|BUILDING|to the even, regular platelets. regular, ironic epitaphs nag e|
+2|Customer#000000002|XSTf4,NCwDVaWNe6tEgvwfmRchLXak|13|23-768-687-3665|121.65|AUTOMOBILE|l accounts. blithely ironic theodolites integrate boldly: caref|
+3|Customer#000000003|MG9kdTD2WBHm|1|11-719-748-3364|7498.12|AUTOMOBILE| deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov|
+4|Customer#000000004|XxVSJsLAGtn|4|14-128-190-5944|2866.83|MACHINERY| requests. final, regular ideas sleep final accou|
+5|Customer#000000005|KvpyuHCplrB84WgAiGV6sYpZq7Tj|3|13-750-942-6364|794.47|HOUSEHOLD|n accounts will have to unwind. foxes cajole accor|
+6|Customer#000000006|sKZz0CsnMD7mp4Xd0YrBvx,LREYKUWAh yVn|20|30-114-968-4951|7638.57|AUTOMOBILE|tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious|
+7|Customer#000000007|TcGe5gaZNgVePxU5kRrvXBfkasDTea|18|28-190-982-9759|9561.95|AUTOMOBILE|ainst the ironic, express theodolites. express, even pinto beans among the exp|
+8|Customer#000000008|I0B10bB0AymmC, 0PrRYBCP1yGJ8xcBPmWhl5|17|27-147-574-9335|6819.74|BUILDING|among the slyly regular theodolites kindle blithely courts. carefully even theodolites haggle slyly along the ide|
+9|Customer#000000009|xKiAFTjUsCuxfeleNqefumTrjS|8|18-338-906-3675|8324.07|FURNITURE|r theodolites according to the requests wake thinly excuses: pending requests haggle furiousl|
+10|Customer#000000010|6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2|5|15-741-346-9870|2753.54|HOUSEHOLD|es regular deposits haggle. fur|
+11|Customer#000000011|PkWS 3HlXqwTuzrKg633BEi|23|33-464-151-3439|-272.60|BUILDING|ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. |
+12|Customer#000000012|9PWKuhzT4Zr1Q|13|23-791-276-1263|3396.49|HOUSEHOLD| to the carefully final braids. blithely regular requests nag. ironic theodolites boost quickly along|
+13|Customer#000000013|nsXQu0oVjD7PM659uC3SRSp|3|13-761-547-5974|3857.34|BUILDING|ounts sleep carefully after the close frays. carefully bold notornis use ironic requests. blithely|
+14|Customer#000000014|KXkletMlL2JQEA |1|11-845-129-3851|5266.30|FURNITURE|, ironic packages across the unus|
+15|Customer#000000015|YtWggXoOLdwdo7b0y,BZaGUQMLJMX1Y,EC,6Dn|23|33-687-542-7601|2788.52|HOUSEHOLD| platelets. regular deposits detect asymptotes. blithely unusual packages nag slyly at the fluf|
+16|Customer#000000016|cYiaeMLZSMAOQ2 d0W,|10|20-781-609-3107|4681.03|FURNITURE|kly silent courts. thinly regular theodolites sleep fluffily after |
+17|Customer#000000017|izrh 6jdqtp2eqdtbkswDD8SG4SzXruMfIXyR7|2|12-970-682-3487|6.34|AUTOMOBILE|packages wake! blithely even pint|
+18|Customer#000000018|3txGO AiuFux3zT0Z9NYaFRnZt|6|16-155-215-1315|5494.43|BUILDING|s sleep. carefully even instructions nag furiously alongside of t|
+19|Customer#000000019|uc,3bHIx84H,wdrmLOjVsiqXCq2tr|18|28-396-526-5053|8914.71|HOUSEHOLD| nag. furiously careful packages are slyly at the accounts. furiously regular in|
+20|Customer#000000020|JrPk8Pqplj4Ne|22|32-957-234-8742|7603.40|FURNITURE|g alongside of the special excuses-- fluffily enticing packages wake |
+21|Customer#000000021|XYmVpr9yAHDEn|8|18-902-614-8344|1428.25|MACHINERY| quickly final accounts integrate blithely furiously u|
+22|Customer#000000022|QI6p41,FNs5k7RZoCCVPUTkUdYpB|3|13-806-545-9701|591.98|MACHINERY|s nod furiously above the furiously ironic ideas. |
+23|Customer#000000023|OdY W13N7Be3OC5MpgfmcYss0Wn6TKT|3|13-312-472-8245|3332.02|HOUSEHOLD|deposits. special deposits cajole slyly. fluffily special deposits about the furiously |
+24|Customer#000000024|HXAFgIAyjxtdqwimt13Y3OZO 4xeLe7U8PqG|13|23-127-851-8031|9255.67|MACHINERY|into beans. fluffily final ideas haggle fluffily|
+25|Customer#000000025|Hp8GyFQgGHFYSilH5tBfe|12|22-603-468-3533|7133.70|FURNITURE|y. accounts sleep ruthlessly according to the regular theodolites. unusual instructions sleep. ironic, final|
+26|Customer#000000026|8ljrc5ZeMl7UciP|22|32-363-455-4837|5182.05|AUTOMOBILE|c requests use furiously ironic requests. slyly ironic dependencies us|
+27|Customer#000000027|IS8GIyxpBrLpMT0u7|3|13-137-193-2709|5679.84|BUILDING| about the carefully ironic pinto beans. accoun|
+28|Customer#000000028|iVyg0daQ,Tha8x2WPWA9m2529m|8|18-774-241-1462|1007.18|FURNITURE| along the regular deposits. furiously final pac|
+29|Customer#000000029|sJ5adtfyAkCK63df2,vF25zyQMVYE34uh|0|10-773-203-7342|7618.27|FURNITURE|its after the carefully final platelets x-ray against |
+30|Customer#000000030|nJDsELGAavU63Jl0c5NKsKfL8rIJQQkQnYL2QJY|1|11-764-165-5076|9321.01|BUILDING|lithely final requests. furiously unusual account|
+31|Customer#000000031|LUACbO0viaAv6eXOAebryDB xjVst|23|33-197-837-7094|5236.89|HOUSEHOLD|s use among the blithely pending depo|
+32|Customer#000000032|jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J|15|25-430-914-2194|3471.53|BUILDING|cial ideas. final, furious requests across the e|
+33|Customer#000000033|qFSlMuLucBmx9xnn5ib2csWUweg D|17|27-375-391-1280|-78.56|AUTOMOBILE|s. slyly regular accounts are furiously. carefully pending requests|
+34|Customer#000000034|Q6G9wZ6dnczmtOx509xgE,M2KV|15|25-344-968-5422|8589.70|HOUSEHOLD|nder against the even, pending accounts. even|
+35|Customer#000000035|TEjWGE4nBzJL2|17|27-566-888-7431|1228.24|HOUSEHOLD|requests. special, express requests nag slyly furiousl|
+36|Customer#000000036|3TvCzjuPzpJ0,DdJ8kW5U|21|31-704-669-5769|4987.27|BUILDING|haggle. enticing, quiet platelets grow quickly bold sheaves. carefully regular acc|
+37|Customer#000000037|7EV4Pwh,3SboctTWt|8|18-385-235-7162|-917.75|FURNITURE|ilent packages are carefully among the deposits. furiousl|
+38|Customer#000000038|a5Ee5e9568R8RLP 2ap7|12|22-306-880-7212|6345.11|HOUSEHOLD|lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin|
+39|Customer#000000039|nnbRg,Pvy33dfkorYE FdeZ60|2|12-387-467-6509|6264.31|AUTOMOBILE|tions. slyly silent excuses slee|
+40|Customer#000000040|gOnGWAyhSV1ofv|3|13-652-915-8939|1335.30|BUILDING|rges impress after the slyly ironic courts. foxes are. blithely |
+41|Customer#000000041|IM9mzmyoxeBmvNw8lA7G3Ydska2nkZF|10|20-917-711-4011|270.95|HOUSEHOLD|ly regular accounts hang bold, silent packages. unusual foxes haggle slyly above the special, final depo|
+42|Customer#000000042|ziSrvyyBke|5|15-416-330-4175|8727.01|BUILDING|ssly according to the pinto beans: carefully special requests across the even, pending accounts wake special|
+43|Customer#000000043|ouSbjHk8lh5fKX3zGso3ZSIj9Aa3PoaFd|19|29-316-665-2897|9904.28|MACHINERY|ial requests: carefully pending foxes detect quickly. carefully final courts cajole quickly. carefully|
+44|Customer#000000044|Oi,dOSPwDu4jo4x,,P85E0dmhZGvNtBwi|16|26-190-260-5375|7315.94|AUTOMOBILE|r requests around the unusual, bold a|
+45|Customer#000000045|4v3OcpFgoOmMG,CbnF,4mdC|9|19-715-298-9917|9983.38|AUTOMOBILE|nto beans haggle slyly alongside of t|
+46|Customer#000000046|eaTXWWm10L9|6|16-357-681-2007|5744.59|AUTOMOBILE|ctions. accounts sleep furiously even requests. regular, regular accounts cajole blithely around the final pa|
+47|Customer#000000047|b0UgocSqEW5 gdVbhNT|2|12-427-271-9466|274.58|BUILDING|ions. express, ironic instructions sleep furiously ironic ideas. furi|
+48|Customer#000000048|0UU iPhBupFvemNB|0|10-508-348-5882|3792.50|BUILDING|re fluffily pending foxes. pending, bold platelets sleep slyly. even platelets cajo|
+49|Customer#000000049|cNgAeX7Fqrdf7HQN9EwjUa4nxT,68L FKAxzl|10|20-908-631-4424|4573.94|FURNITURE|nusual foxes! fluffily pending packages maintain to the regular |
+50|Customer#000000050|9SzDYlkzxByyJ1QeTI o|6|16-658-112-3221|4266.13|MACHINERY|ts. furiously ironic accounts cajole furiously slyly ironic dinos.|
+51|Customer#000000051|uR,wEaiTvo4|12|22-344-885-4251|855.87|FURNITURE|eposits. furiously regular requests integrate carefully packages. furious|
+52|Customer#000000052|7 QOqGqqSy9jfV51BC71jcHJSD0|11|21-186-284-5998|5630.28|HOUSEHOLD|ic platelets use evenly even accounts. stealthy theodolites cajole furiou|
+53|Customer#000000053|HnaxHzTfFTZs8MuCpJyTbZ47Cm4wFOOgib|15|25-168-852-5363|4113.64|HOUSEHOLD|ar accounts are. even foxes are blithely. fluffily pending deposits boost|
+54|Customer#000000054|,k4vf 5vECGWFy,hosTE,|4|14-776-370-4745|868.90|AUTOMOBILE|sual, silent accounts. furiously express accounts cajole special deposits. final, final accounts use furi|
+55|Customer#000000055|zIRBR4KNEl HzaiV3a i9n6elrxzDEh8r8pDom|10|20-180-440-8525|4572.11|MACHINERY|ully unusual packages wake bravely bold packages. unusual requests boost deposits! blithely ironic packages ab|
+56|Customer#000000056|BJYZYJQk4yD5B|10|20-895-685-6920|6530.86|FURNITURE|. notornis wake carefully. carefully fluffy requests are furiously even accounts. slyly expre|
+57|Customer#000000057|97XYbsuOPRXPWU|21|31-835-306-1650|4151.93|AUTOMOBILE|ove the carefully special packages. even, unusual deposits sleep slyly pend|
+58|Customer#000000058|g9ap7Dk1Sv9fcXEWjpMYpBZIRUohi T|13|23-244-493-2508|6478.46|HOUSEHOLD|ideas. ironic ideas affix furiously express, final instructions. regular excuses use quickly e|
+59|Customer#000000059|zLOCP0wh92OtBihgspOGl4|1|11-355-584-3112|3458.60|MACHINERY|ously final packages haggle blithely after the express deposits. furiou|
+60|Customer#000000060|FyodhjwMChsZmUz7Jz0H|12|22-480-575-5866|2741.87|MACHINERY|latelets. blithely unusual courts boost furiously about the packages. blithely final instruct|
+61|Customer#000000061|9kndve4EAJxhg3veF BfXr7AqOsT39o gtqjaYE|17|27-626-559-8599|1536.24|FURNITURE|egular packages shall have to impress along the |
+62|Customer#000000062|upJK2Dnw13,|7|17-361-978-7059|595.61|MACHINERY|kly special dolphins. pinto beans are slyly. quickly regular accounts are furiously a|
+63|Customer#000000063|IXRSpVWWZraKII|21|31-952-552-9584|9331.13|AUTOMOBILE|ithely even accounts detect slyly above the fluffily ir|
+64|Customer#000000064|MbCeGY20kaKK3oalJD,OT|3|13-558-731-7204|-646.64|BUILDING|structions after the quietly ironic theodolites cajole be|
+65|Customer#000000065|RGT yzQ0y4l0H90P783LG4U95bXQFDRXbWa1sl,X|23|33-733-623-5267|8795.16|AUTOMOBILE|y final foxes serve carefully. theodolites are carefully. pending i|
+66|Customer#000000066|XbsEqXH1ETbJYYtA1A|22|32-213-373-5094|242.77|HOUSEHOLD|le slyly accounts. carefully silent packages benea|
+67|Customer#000000067|rfG0cOgtr5W8 xILkwp9fpCS8|9|19-403-114-4356|8166.59|MACHINERY|indle furiously final, even theodo|
+68|Customer#000000068|o8AibcCRkXvQFh8hF,7o|12|22-918-832-2411|6853.37|HOUSEHOLD| pending pinto beans impress realms. final dependencies |
+69|Customer#000000069|Ltx17nO9Wwhtdbe9QZVxNgP98V7xW97uvSH1prEw|9|19-225-978-5670|1709.28|HOUSEHOLD|thely final ideas around the quickly final dependencies affix carefully quickly final theodolites. final accounts c|
+70|Customer#000000070|mFowIuhnHjp2GjCiYYavkW kUwOjIaTCQ|22|32-828-107-2832|4867.52|FURNITURE|fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be|
+71|Customer#000000071|TlGalgdXWBmMV,6agLyWYDyIz9MKzcY8gl,w6t1B|7|17-710-812-5403|-611.19|HOUSEHOLD|g courts across the regular, final pinto beans are blithely pending ac|
+72|Customer#000000072|putjlmskxE,zs,HqeIA9Wqu7dhgH5BVCwDwHHcf|2|12-759-144-9689|-362.86|FURNITURE|ithely final foxes sleep always quickly bold accounts. final wat|
+73|Customer#000000073|8IhIxreu4Ug6tt5mog4|0|10-473-439-3214|4288.50|BUILDING|usual, unusual packages sleep busily along the furiou|
+74|Customer#000000074|IkJHCA3ZThF7qL7VKcrU nRLl,kylf |4|14-199-862-7209|2764.43|MACHINERY|onic accounts. blithely slow packages would haggle carefully. qui|
+75|Customer#000000075|Dh 6jZ,cwxWLKQfRKkiGrzv6pm|18|28-247-803-9025|6684.10|AUTOMOBILE| instructions cajole even, even deposits. finally bold deposits use above the even pains. slyl|
+76|Customer#000000076|m3sbCvjMOHyaOofH,e UkGPtqc4|0|10-349-718-3044|5745.33|FURNITURE|pecial deposits. ironic ideas boost blithely according to the closely ironic theodolites! furiously final deposits n|
+77|Customer#000000077|4tAE5KdMFGD4byHtXF92vx|17|27-269-357-4674|1738.87|BUILDING|uffily silent requests. carefully ironic asymptotes among the ironic hockey players are carefully bli|
+78|Customer#000000078|HBOta,ZNqpg3U2cSL0kbrftkPwzX|9|19-960-700-9191|7136.97|FURNITURE|ests. blithely bold pinto beans h|
+79|Customer#000000079|n5hH2ftkVRwW8idtD,BmM2|15|25-147-850-4166|5121.28|MACHINERY|es. packages haggle furiously. regular, special requests poach after the quickly express ideas. blithely pending re|
+80|Customer#000000080|K,vtXp8qYB |0|10-267-172-7101|7383.53|FURNITURE|tect among the dependencies. bold accounts engage closely even pinto beans. ca|
+81|Customer#000000081|SH6lPA7JiiNC6dNTrR|20|30-165-277-3269|2023.71|BUILDING|r packages. fluffily ironic requests cajole fluffily. ironically regular theodolit|
+82|Customer#000000082|zhG3EZbap4c992Gj3bK,3Ne,Xn|18|28-159-442-5305|9468.34|AUTOMOBILE|s wake. bravely regular accounts are furiously. regula|
+83|Customer#000000083|HnhTNB5xpnSF20JBH4Ycs6psVnkC3RDf|22|32-817-154-4122|6463.51|BUILDING|ccording to the quickly bold warhorses. final, regular foxes integrate carefully. bold packages nag blithely ev|
+84|Customer#000000084|lpXz6Fwr9945rnbtMc8PlueilS1WmASr CB|11|21-546-818-3802|5174.71|FURNITURE|ly blithe foxes. special asymptotes haggle blithely against the furiously regular depo|
+85|Customer#000000085|siRerlDwiolhYR 8FgksoezycLj|5|15-745-585-8219|3386.64|FURNITURE|ronic ideas use above the slowly pendin|
+86|Customer#000000086|US6EGGHXbTTXPL9SBsxQJsuvy|0|10-677-951-2353|3306.32|HOUSEHOLD|quests. pending dugouts are carefully aroun|
+87|Customer#000000087|hgGhHVSWQl 6jZ6Ev|23|33-869-884-7053|6327.54|FURNITURE|hely ironic requests integrate according to the ironic accounts. slyly regular pla|
+88|Customer#000000088|wtkjBN9eyrFuENSMmMFlJ3e7jE5KXcg|16|26-516-273-2566|8031.44|AUTOMOBILE|s are quickly above the quickly ironic instructions; even requests about the carefully final deposi|
+89|Customer#000000089|dtR, y9JQWUO6FoJExyp8whOU|14|24-394-451-5404|1530.76|FURNITURE|counts are slyly beyond the slyly final accounts. quickly final ideas wake. r|
+90|Customer#000000090|QxCzH7VxxYUWwfL7|16|26-603-491-1238|7354.23|BUILDING|sly across the furiously even |
+91|Customer#000000091|S8OMYFrpHwoNHaGBeuS6E 6zhHGZiprw1b7 q|8|18-239-400-3677|4643.14|AUTOMOBILE|onic accounts. fluffily silent pinto beans boost blithely according to the fluffily exp|
+92|Customer#000000092|obP PULk2LH LqNF,K9hcbNqnLAkJVsl5xqSrY,|2|12-446-416-8471|1182.91|MACHINERY|. pinto beans hang slyly final deposits. ac|
+93|Customer#000000093|EHXBr2QGdh|7|17-359-388-5266|2182.52|MACHINERY|press deposits. carefully regular platelets r|
+94|Customer#000000094|IfVNIN9KtkScJ9dUjK3Pg5gY1aFeaXewwf|9|19-953-499-8833|5500.11|HOUSEHOLD|latelets across the bold, final requests sleep according to the fluffily bold accounts. unusual deposits amon|
+95|Customer#000000095|EU0xvmWvOmUUn5J,2z85DQyG7QCJ9Xq7|15|25-923-255-2929|5327.38|MACHINERY|ithely. ruthlessly final requests wake slyly alongside of the furiously silent pinto beans. even the|
+96|Customer#000000096|vWLOrmXhRR|8|18-422-845-1202|6323.92|AUTOMOBILE|press requests believe furiously. carefully final instructions snooze carefully. |
+97|Customer#000000097|OApyejbhJG,0Iw3j rd1M|17|27-588-919-5638|2164.48|AUTOMOBILE|haggle slyly. bold, special ideas are blithely above the thinly bold theo|
+98|Customer#000000098|7yiheXNSpuEAwbswDW|12|22-885-845-6889|-551.37|BUILDING|ages. furiously pending accounts are quickly carefully final foxes: busily pe|
+99|Customer#000000099|szsrOiPtCHVS97Lt|15|25-515-237-9232|4088.65|HOUSEHOLD|cajole slyly about the regular theodolites! furiously bold requests nag along the pending, regular packages. somas|
+100|Customer#000000100|fptUABXcmkC5Wx|20|30-749-445-4907|9889.89|FURNITURE|was furiously fluffily quiet deposits. silent, pending requests boost against |
+101|Customer#000000101|sMmL2rNeHDltovSm Y|2|12-514-298-3699|7470.96|MACHINERY| sleep. pending packages detect slyly ironic pack|
+102|Customer#000000102|UAtflJ06 fn9zBfKjInkQZlWtqaA|19|29-324-978-8538|8462.17|BUILDING|ously regular dependencies nag among the furiously express dinos. blithely final|
+103|Customer#000000103|8KIsQX4LJ7QMsj6DrtFtXu0nUEdV,8a|9|19-216-107-2107|2757.45|BUILDING|furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl|
+104|Customer#000000104|9mcCK L7rt0SwiYtrbO88DiZS7U d7M|10|20-966-284-8065|-588.38|FURNITURE|rate carefully slyly special pla|
+105|Customer#000000105|4iSJe4L SPjg7kJj98Yz3z0B|10|20-793-553-6417|9091.82|MACHINERY|l pains cajole even accounts. quietly final instructi|
+106|Customer#000000106|xGCOEAUjUNG|1|11-751-989-4627|3288.42|MACHINERY|lose slyly. ironic accounts along the evenly regular theodolites wake about the special, final gifts. |
+107|Customer#000000107|Zwg64UZ,q7GRqo3zm7P1tZIRshBDz|15|25-336-529-9919|2514.15|AUTOMOBILE|counts cajole slyly. regular requests wake. furiously regular deposits about the blithely final fo|
+108|Customer#000000108|GPoeEvpKo1|5|15-908-619-7526|2259.38|BUILDING|refully ironic deposits sleep. regular, unusual requests wake slyly|
+109|Customer#000000109|OOOkYBgCMzgMQXUmkocoLb56rfrdWp2NE2c|16|26-992-422-8153|-716.10|BUILDING|es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou|
+110|Customer#000000110|mymPfgphaYXNYtk|10|20-893-536-2069|7462.99|AUTOMOBILE|nto beans cajole around the even, final deposits. quickly bold packages according to the furiously regular dept|
+111|Customer#000000111|CBSbPyOWRorloj2TBvrK9qp9tHBs|22|32-582-283-7528|6505.26|MACHINERY|ly unusual instructions detect fluffily special deposits-- theodolites nag carefully during the ironic dependencies|
+112|Customer#000000112|RcfgG3bO7QeCnfjqJT1|19|29-233-262-8382|2953.35|FURNITURE|rmanently unusual multipliers. blithely ruthless deposits are furiously along the|
+113|Customer#000000113|eaOl5UBXIvdY57rglaIzqvfPD,MYfK|12|22-302-930-4756|2912.00|BUILDING|usly regular theodolites boost furiously doggedly pending instructio|
+114|Customer#000000114|xAt 5f5AlFIU|14|24-805-212-7646|1027.46|FURNITURE|der the carefully express theodolites are after the packages. packages are. bli|
+115|Customer#000000115|0WFt1IXENmUT2BgbsB0ShVKJZt0HCBCbFl0aHc|8|18-971-699-1843|7508.92|HOUSEHOLD|sits haggle above the carefully ironic theodolite|
+116|Customer#000000116|yCuVxIgsZ3,qyK2rloThy3u|16|26-632-309-5792|8403.99|BUILDING|as. quickly final sauternes haggle slyly carefully even packages. brave, ironic pinto beans are above the furious|
+117|Customer#000000117|uNhM,PzsRA3S,5Y Ge5Npuhi|24|34-403-631-3505|3950.83|FURNITURE|affix. instructions are furiously sl|
+118|Customer#000000118|OVnFuHygK9wx3xpg8|18|28-639-943-7051|3582.37|AUTOMOBILE|uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep|
+119|Customer#000000119|M1ETOIecuvH8DtM0Y0nryXfW|7|17-697-919-8406|3930.35|FURNITURE|express ideas. blithely ironic foxes thrash. special acco|
+120|Customer#000000120|zBNna00AEInqyO1|12|22-291-534-1571|363.75|MACHINERY| quickly. slyly ironic requests cajole blithely furiously final dependen|
+121|Customer#000000121|tv nCR2YKupGN73mQudO|17|27-411-990-2959|6428.32|BUILDING|uriously stealthy ideas. carefully final courts use carefully|
+122|Customer#000000122|yp5slqoNd26lAENZW3a67wSfXA6hTF|3|13-702-694-4520|7865.46|HOUSEHOLD| the special packages hinder blithely around the permanent requests. bold depos|
+123|Customer#000000123|YsOnaaER8MkvK5cpf4VSlq|5|15-817-151-1168|5897.83|BUILDING|ependencies. regular, ironic requests are fluffily regu|
+124|Customer#000000124|aTbyVAW5tCd,v09O|18|28-183-750-7809|1842.49|AUTOMOBILE|le fluffily even dependencies. quietly s|
+125|Customer#000000125|,wSZXdVR xxIIfm9s8ITyLl3kgjT6UC07GY0Y|19|29-261-996-3120|-234.12|FURNITURE|x-ray finally after the packages? regular requests c|
+126|Customer#000000126|ha4EHmbx3kg DYCsP6DFeUOmavtQlHhcfaqr|22|32-755-914-7592|1001.39|HOUSEHOLD|s about the even instructions boost carefully furiously ironic pearls. ruthless, |
+127|Customer#000000127|Xyge4DX2rXKxXyye1Z47LeLVEYMLf4Bfcj|21|31-101-672-2951|9280.71|MACHINERY|ic, unusual theodolites nod silently after the final, ironic instructions: pending r|
+128|Customer#000000128|AmKUMlJf2NRHcKGmKjLS|4|14-280-874-8044|-986.96|HOUSEHOLD|ing packages integrate across the slyly unusual dugouts. blithely silent ideas sublate carefully. blithely expr|
+129|Customer#000000129|q7m7rbMM0BpaCdmxloCgBDRCleXsXkdD8kf|7|17-415-148-7416|9127.27|HOUSEHOLD| unusual deposits boost carefully furiously silent ideas. pending accounts cajole slyly across|
+130|Customer#000000130|RKPx2OfZy0Vn 8wGWZ7F2EAvmMORl1k8iH|9|19-190-993-9281|5073.58|HOUSEHOLD|ix slowly. express packages along the furiously ironic requests integrate daringly deposits. fur|
+131|Customer#000000131|jyN6lAjb1FtH10rMC,XzlWyCBrg75|11|21-840-210-3572|8595.53|HOUSEHOLD|jole special packages. furiously final dependencies about the furiously speci|
+132|Customer#000000132|QM5YabAsTLp9|4|14-692-150-9717|162.57|HOUSEHOLD|uickly carefully special theodolites. carefully regular requests against the blithely unusual instructions |
+133|Customer#000000133|IMCuXdpIvdkYO92kgDGuyHgojcUs88p|17|27-408-997-8430|2314.67|AUTOMOBILE|t packages. express pinto beans are blithely along the unusual, even theodolites. silent packages use fu|
+134|Customer#000000134|sUiZ78QCkTQPICKpA9OBzkUp2FM|11|21-200-159-5932|4608.90|BUILDING|yly fluffy foxes boost final ideas. b|
+135|Customer#000000135|oZK,oC0 fdEpqUML|19|29-399-293-6241|8732.91|FURNITURE| the slyly final accounts. deposits cajole carefully. carefully sly packag|
+136|Customer#000000136|QoLsJ0v5C1IQbh,DS1|7|17-501-210-4726|-842.39|FURNITURE|ackages sleep ironic, final courts. even requests above the blithely bold requests g|
+137|Customer#000000137|cdW91p92rlAEHgJafqYyxf1Q|16|26-777-409-5654|7838.30|HOUSEHOLD|carefully regular theodolites use. silent dolphins cajo|
+138|Customer#000000138|5uyLAeY7HIGZqtu66Yn08f|5|15-394-860-4589|430.59|MACHINERY|ts doze on the busy ideas. regular|
+139|Customer#000000139|3ElvBwudHKL02732YexGVFVt |9|19-140-352-1403|7897.78|MACHINERY|nstructions. quickly ironic ideas are carefully. bold, |
+140|Customer#000000140|XRqEPiKgcETII,iOLDZp5jA|4|14-273-885-6505|9963.15|MACHINERY|ies detect slyly ironic accounts. slyly ironic theodolites hag|
+141|Customer#000000141|5IW,WROVnikc3l7DwiUDGQNGsLBGOL6Dc0|1|11-936-295-6204|6706.14|FURNITURE|packages nag furiously. carefully unusual accounts snooze according to the fluffily regular pinto beans. slyly spec|
+142|Customer#000000142|AnJ5lxtLjioClr2khl9pb8NLxG2,|9|19-407-425-2584|2209.81|AUTOMOBILE|. even, express theodolites upo|
+143|Customer#000000143|681r22uL452zqk 8By7I9o9enQfx0|16|26-314-406-7725|2186.50|MACHINERY|across the blithely unusual requests haggle theodo|
+144|Customer#000000144|VxYZ3ebhgbltnetaGjNC8qCccjYU05 fePLOno8y|1|11-717-379-4478|6417.31|MACHINERY|ges. slyly regular accounts are slyly. bold, idle reque|
+145|Customer#000000145|kQjHmt2kcec cy3hfMh969u|13|23-562-444-8454|9748.93|HOUSEHOLD|ests? express, express instructions use. blithely fina|
+146|Customer#000000146|GdxkdXG9u7iyI1,,y5tq4ZyrcEy|3|13-835-723-3223|3328.68|FURNITURE|ffily regular dinos are slyly unusual requests. slyly specia|
+147|Customer#000000147|6VvIwbVdmcsMzuu,C84GtBWPaipGfi7DV|18|28-803-187-4335|8071.40|AUTOMOBILE|ress packages above the blithely regular packages sleep fluffily blithely ironic accounts. |
+148|Customer#000000148|BhSPlEWGvIJyT9swk vCWE|11|21-562-498-6636|2135.60|HOUSEHOLD|ing to the carefully ironic requests. carefully regular dependencies about the theodolites wake furious|
+149|Customer#000000149|3byTHCp2mNLPigUrrq|19|29-797-439-6760|8959.65|AUTOMOBILE|al instructions haggle against the slyly bold w|
+150|Customer#000000150|zeoGShTjCwGPplOWFkLURrh41O0AZ8dwNEEN4 |18|28-328-564-7630|3849.48|MACHINERY|ole blithely among the furiously pending packages. furiously bold ideas wake fluffily ironic idea|
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/expected/part-0 b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/expected/part-0
new file mode 100755
index 0000000..ce3b00c
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/expected/part-0
@@ -0,0 +1,150 @@
+100|Customer#000000100|fptUABXcmkC5Wx|20|30-749-445-4907|9889.89|FURNITURE|was furiously fluffily quiet deposits. silent, pending requests boost against |
+101|Customer#000000101|sMmL2rNeHDltovSm Y|2|12-514-298-3699|7470.96|MACHINERY| sleep. pending packages detect slyly ironic pack|
+102|Customer#000000102|UAtflJ06 fn9zBfKjInkQZlWtqaA|19|29-324-978-8538|8462.17|BUILDING|ously regular dependencies nag among the furiously express dinos. blithely final|
+103|Customer#000000103|8KIsQX4LJ7QMsj6DrtFtXu0nUEdV,8a|9|19-216-107-2107|2757.45|BUILDING|furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl|
+104|Customer#000000104|9mcCK L7rt0SwiYtrbO88DiZS7U d7M|10|20-966-284-8065|-588.38|FURNITURE|rate carefully slyly special pla|
+105|Customer#000000105|4iSJe4L SPjg7kJj98Yz3z0B|10|20-793-553-6417|9091.82|MACHINERY|l pains cajole even accounts. quietly final instructi|
+106|Customer#000000106|xGCOEAUjUNG|1|11-751-989-4627|3288.42|MACHINERY|lose slyly. ironic accounts along the evenly regular theodolites wake about the special, final gifts. |
+107|Customer#000000107|Zwg64UZ,q7GRqo3zm7P1tZIRshBDz|15|25-336-529-9919|2514.15|AUTOMOBILE|counts cajole slyly. regular requests wake. furiously regular deposits about the blithely final fo|
+108|Customer#000000108|GPoeEvpKo1|5|15-908-619-7526|2259.38|BUILDING|refully ironic deposits sleep. regular, unusual requests wake slyly|
+109|Customer#000000109|OOOkYBgCMzgMQXUmkocoLb56rfrdWp2NE2c|16|26-992-422-8153|-716.10|BUILDING|es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou|
+10|Customer#000000010|6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2|5|15-741-346-9870|2753.54|HOUSEHOLD|es regular deposits haggle. fur|
+110|Customer#000000110|mymPfgphaYXNYtk|10|20-893-536-2069|7462.99|AUTOMOBILE|nto beans cajole around the even, final deposits. quickly bold packages according to the furiously regular dept|
+111|Customer#000000111|CBSbPyOWRorloj2TBvrK9qp9tHBs|22|32-582-283-7528|6505.26|MACHINERY|ly unusual instructions detect fluffily special deposits-- theodolites nag carefully during the ironic dependencies|
+112|Customer#000000112|RcfgG3bO7QeCnfjqJT1|19|29-233-262-8382|2953.35|FURNITURE|rmanently unusual multipliers. blithely ruthless deposits are furiously along the|
+113|Customer#000000113|eaOl5UBXIvdY57rglaIzqvfPD,MYfK|12|22-302-930-4756|2912.00|BUILDING|usly regular theodolites boost furiously doggedly pending instructio|
+114|Customer#000000114|xAt 5f5AlFIU|14|24-805-212-7646|1027.46|FURNITURE|der the carefully express theodolites are after the packages. packages are. bli|
+115|Customer#000000115|0WFt1IXENmUT2BgbsB0ShVKJZt0HCBCbFl0aHc|8|18-971-699-1843|7508.92|HOUSEHOLD|sits haggle above the carefully ironic theodolite|
+116|Customer#000000116|yCuVxIgsZ3,qyK2rloThy3u|16|26-632-309-5792|8403.99|BUILDING|as. quickly final sauternes haggle slyly carefully even packages. brave, ironic pinto beans are above the furious|
+117|Customer#000000117|uNhM,PzsRA3S,5Y Ge5Npuhi|24|34-403-631-3505|3950.83|FURNITURE|affix. instructions are furiously sl|
+118|Customer#000000118|OVnFuHygK9wx3xpg8|18|28-639-943-7051|3582.37|AUTOMOBILE|uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep|
+119|Customer#000000119|M1ETOIecuvH8DtM0Y0nryXfW|7|17-697-919-8406|3930.35|FURNITURE|express ideas. blithely ironic foxes thrash. special acco|
+11|Customer#000000011|PkWS 3HlXqwTuzrKg633BEi|23|33-464-151-3439|-272.60|BUILDING|ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. |
+120|Customer#000000120|zBNna00AEInqyO1|12|22-291-534-1571|363.75|MACHINERY| quickly. slyly ironic requests cajole blithely furiously final dependen|
+121|Customer#000000121|tv nCR2YKupGN73mQudO|17|27-411-990-2959|6428.32|BUILDING|uriously stealthy ideas. carefully final courts use carefully|
+122|Customer#000000122|yp5slqoNd26lAENZW3a67wSfXA6hTF|3|13-702-694-4520|7865.46|HOUSEHOLD| the special packages hinder blithely around the permanent requests. bold depos|
+123|Customer#000000123|YsOnaaER8MkvK5cpf4VSlq|5|15-817-151-1168|5897.83|BUILDING|ependencies. regular, ironic requests are fluffily regu|
+124|Customer#000000124|aTbyVAW5tCd,v09O|18|28-183-750-7809|1842.49|AUTOMOBILE|le fluffily even dependencies. quietly s|
+125|Customer#000000125|,wSZXdVR xxIIfm9s8ITyLl3kgjT6UC07GY0Y|19|29-261-996-3120|-234.12|FURNITURE|x-ray finally after the packages? regular requests c|
+126|Customer#000000126|ha4EHmbx3kg DYCsP6DFeUOmavtQlHhcfaqr|22|32-755-914-7592|1001.39|HOUSEHOLD|s about the even instructions boost carefully furiously ironic pearls. ruthless, |
+127|Customer#000000127|Xyge4DX2rXKxXyye1Z47LeLVEYMLf4Bfcj|21|31-101-672-2951|9280.71|MACHINERY|ic, unusual theodolites nod silently after the final, ironic instructions: pending r|
+128|Customer#000000128|AmKUMlJf2NRHcKGmKjLS|4|14-280-874-8044|-986.96|HOUSEHOLD|ing packages integrate across the slyly unusual dugouts. blithely silent ideas sublate carefully. blithely expr|
+129|Customer#000000129|q7m7rbMM0BpaCdmxloCgBDRCleXsXkdD8kf|7|17-415-148-7416|9127.27|HOUSEHOLD| unusual deposits boost carefully furiously silent ideas. pending accounts cajole slyly across|
+12|Customer#000000012|9PWKuhzT4Zr1Q|13|23-791-276-1263|3396.49|HOUSEHOLD| to the carefully final braids. blithely regular requests nag. ironic theodolites boost quickly along|
+130|Customer#000000130|RKPx2OfZy0Vn 8wGWZ7F2EAvmMORl1k8iH|9|19-190-993-9281|5073.58|HOUSEHOLD|ix slowly. express packages along the furiously ironic requests integrate daringly deposits. fur|
+131|Customer#000000131|jyN6lAjb1FtH10rMC,XzlWyCBrg75|11|21-840-210-3572|8595.53|HOUSEHOLD|jole special packages. furiously final dependencies about the furiously speci|
+132|Customer#000000132|QM5YabAsTLp9|4|14-692-150-9717|162.57|HOUSEHOLD|uickly carefully special theodolites. carefully regular requests against the blithely unusual instructions |
+133|Customer#000000133|IMCuXdpIvdkYO92kgDGuyHgojcUs88p|17|27-408-997-8430|2314.67|AUTOMOBILE|t packages. express pinto beans are blithely along the unusual, even theodolites. silent packages use fu|
+134|Customer#000000134|sUiZ78QCkTQPICKpA9OBzkUp2FM|11|21-200-159-5932|4608.90|BUILDING|yly fluffy foxes boost final ideas. b|
+135|Customer#000000135|oZK,oC0 fdEpqUML|19|29-399-293-6241|8732.91|FURNITURE| the slyly final accounts. deposits cajole carefully. carefully sly packag|
+136|Customer#000000136|QoLsJ0v5C1IQbh,DS1|7|17-501-210-4726|-842.39|FURNITURE|ackages sleep ironic, final courts. even requests above the blithely bold requests g|
+137|Customer#000000137|cdW91p92rlAEHgJafqYyxf1Q|16|26-777-409-5654|7838.30|HOUSEHOLD|carefully regular theodolites use. silent dolphins cajo|
+138|Customer#000000138|5uyLAeY7HIGZqtu66Yn08f|5|15-394-860-4589|430.59|MACHINERY|ts doze on the busy ideas. regular|
+139|Customer#000000139|3ElvBwudHKL02732YexGVFVt |9|19-140-352-1403|7897.78|MACHINERY|nstructions. quickly ironic ideas are carefully. bold, |
+13|Customer#000000013|nsXQu0oVjD7PM659uC3SRSp|3|13-761-547-5974|3857.34|BUILDING|ounts sleep carefully after the close frays. carefully bold notornis use ironic requests. blithely|
+140|Customer#000000140|XRqEPiKgcETII,iOLDZp5jA|4|14-273-885-6505|9963.15|MACHINERY|ies detect slyly ironic accounts. slyly ironic theodolites hag|
+141|Customer#000000141|5IW,WROVnikc3l7DwiUDGQNGsLBGOL6Dc0|1|11-936-295-6204|6706.14|FURNITURE|packages nag furiously. carefully unusual accounts snooze according to the fluffily regular pinto beans. slyly spec|
+142|Customer#000000142|AnJ5lxtLjioClr2khl9pb8NLxG2,|9|19-407-425-2584|2209.81|AUTOMOBILE|. even, express theodolites upo|
+143|Customer#000000143|681r22uL452zqk 8By7I9o9enQfx0|16|26-314-406-7725|2186.50|MACHINERY|across the blithely unusual requests haggle theodo|
+144|Customer#000000144|VxYZ3ebhgbltnetaGjNC8qCccjYU05 fePLOno8y|1|11-717-379-4478|6417.31|MACHINERY|ges. slyly regular accounts are slyly. bold, idle reque|
+145|Customer#000000145|kQjHmt2kcec cy3hfMh969u|13|23-562-444-8454|9748.93|HOUSEHOLD|ests? express, express instructions use. blithely fina|
+146|Customer#000000146|GdxkdXG9u7iyI1,,y5tq4ZyrcEy|3|13-835-723-3223|3328.68|FURNITURE|ffily regular dinos are slyly unusual requests. slyly specia|
+147|Customer#000000147|6VvIwbVdmcsMzuu,C84GtBWPaipGfi7DV|18|28-803-187-4335|8071.40|AUTOMOBILE|ress packages above the blithely regular packages sleep fluffily blithely ironic accounts. |
+148|Customer#000000148|BhSPlEWGvIJyT9swk vCWE|11|21-562-498-6636|2135.60|HOUSEHOLD|ing to the carefully ironic requests. carefully regular dependencies about the theodolites wake furious|
+149|Customer#000000149|3byTHCp2mNLPigUrrq|19|29-797-439-6760|8959.65|AUTOMOBILE|al instructions haggle against the slyly bold w|
+14|Customer#000000014|KXkletMlL2JQEA |1|11-845-129-3851|5266.30|FURNITURE|, ironic packages across the unus|
+150|Customer#000000150|zeoGShTjCwGPplOWFkLURrh41O0AZ8dwNEEN4 |18|28-328-564-7630|3849.48|MACHINERY|ole blithely among the furiously pending packages. furiously bold ideas wake fluffily ironic idea|
+15|Customer#000000015|YtWggXoOLdwdo7b0y,BZaGUQMLJMX1Y,EC,6Dn|23|33-687-542-7601|2788.52|HOUSEHOLD| platelets. regular deposits detect asymptotes. blithely unusual packages nag slyly at the fluf|
+16|Customer#000000016|cYiaeMLZSMAOQ2 d0W,|10|20-781-609-3107|4681.03|FURNITURE|kly silent courts. thinly regular theodolites sleep fluffily after |
+17|Customer#000000017|izrh 6jdqtp2eqdtbkswDD8SG4SzXruMfIXyR7|2|12-970-682-3487|6.34|AUTOMOBILE|packages wake! blithely even pint|
+18|Customer#000000018|3txGO AiuFux3zT0Z9NYaFRnZt|6|16-155-215-1315|5494.43|BUILDING|s sleep. carefully even instructions nag furiously alongside of t|
+19|Customer#000000019|uc,3bHIx84H,wdrmLOjVsiqXCq2tr|18|28-396-526-5053|8914.71|HOUSEHOLD| nag. furiously careful packages are slyly at the accounts. furiously regular in|
+1|Customer#000000001|IVhzIApeRb ot,c,E|15|25-989-741-2988|711.56|BUILDING|to the even, regular platelets. regular, ironic epitaphs nag e|
+20|Customer#000000020|JrPk8Pqplj4Ne|22|32-957-234-8742|7603.40|FURNITURE|g alongside of the special excuses-- fluffily enticing packages wake |
+21|Customer#000000021|XYmVpr9yAHDEn|8|18-902-614-8344|1428.25|MACHINERY| quickly final accounts integrate blithely furiously u|
+22|Customer#000000022|QI6p41,FNs5k7RZoCCVPUTkUdYpB|3|13-806-545-9701|591.98|MACHINERY|s nod furiously above the furiously ironic ideas. |
+23|Customer#000000023|OdY W13N7Be3OC5MpgfmcYss0Wn6TKT|3|13-312-472-8245|3332.02|HOUSEHOLD|deposits. special deposits cajole slyly. fluffily special deposits about the furiously |
+24|Customer#000000024|HXAFgIAyjxtdqwimt13Y3OZO 4xeLe7U8PqG|13|23-127-851-8031|9255.67|MACHINERY|into beans. fluffily final ideas haggle fluffily|
+25|Customer#000000025|Hp8GyFQgGHFYSilH5tBfe|12|22-603-468-3533|7133.70|FURNITURE|y. accounts sleep ruthlessly according to the regular theodolites. unusual instructions sleep. ironic, final|
+26|Customer#000000026|8ljrc5ZeMl7UciP|22|32-363-455-4837|5182.05|AUTOMOBILE|c requests use furiously ironic requests. slyly ironic dependencies us|
+27|Customer#000000027|IS8GIyxpBrLpMT0u7|3|13-137-193-2709|5679.84|BUILDING| about the carefully ironic pinto beans. accoun|
+28|Customer#000000028|iVyg0daQ,Tha8x2WPWA9m2529m|8|18-774-241-1462|1007.18|FURNITURE| along the regular deposits. furiously final pac|
+29|Customer#000000029|sJ5adtfyAkCK63df2,vF25zyQMVYE34uh|0|10-773-203-7342|7618.27|FURNITURE|its after the carefully final platelets x-ray against |
+2|Customer#000000002|XSTf4,NCwDVaWNe6tEgvwfmRchLXak|13|23-768-687-3665|121.65|AUTOMOBILE|l accounts. blithely ironic theodolites integrate boldly: caref|
+30|Customer#000000030|nJDsELGAavU63Jl0c5NKsKfL8rIJQQkQnYL2QJY|1|11-764-165-5076|9321.01|BUILDING|lithely final requests. furiously unusual account|
+31|Customer#000000031|LUACbO0viaAv6eXOAebryDB xjVst|23|33-197-837-7094|5236.89|HOUSEHOLD|s use among the blithely pending depo|
+32|Customer#000000032|jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J|15|25-430-914-2194|3471.53|BUILDING|cial ideas. final, furious requests across the e|
+33|Customer#000000033|qFSlMuLucBmx9xnn5ib2csWUweg D|17|27-375-391-1280|-78.56|AUTOMOBILE|s. slyly regular accounts are furiously. carefully pending requests|
+34|Customer#000000034|Q6G9wZ6dnczmtOx509xgE,M2KV|15|25-344-968-5422|8589.70|HOUSEHOLD|nder against the even, pending accounts. even|
+35|Customer#000000035|TEjWGE4nBzJL2|17|27-566-888-7431|1228.24|HOUSEHOLD|requests. special, express requests nag slyly furiousl|
+36|Customer#000000036|3TvCzjuPzpJ0,DdJ8kW5U|21|31-704-669-5769|4987.27|BUILDING|haggle. enticing, quiet platelets grow quickly bold sheaves. carefully regular acc|
+37|Customer#000000037|7EV4Pwh,3SboctTWt|8|18-385-235-7162|-917.75|FURNITURE|ilent packages are carefully among the deposits. furiousl|
+38|Customer#000000038|a5Ee5e9568R8RLP 2ap7|12|22-306-880-7212|6345.11|HOUSEHOLD|lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin|
+39|Customer#000000039|nnbRg,Pvy33dfkorYE FdeZ60|2|12-387-467-6509|6264.31|AUTOMOBILE|tions. slyly silent excuses slee|
+3|Customer#000000003|MG9kdTD2WBHm|1|11-719-748-3364|7498.12|AUTOMOBILE| deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov|
+40|Customer#000000040|gOnGWAyhSV1ofv|3|13-652-915-8939|1335.30|BUILDING|rges impress after the slyly ironic courts. foxes are. blithely |
+41|Customer#000000041|IM9mzmyoxeBmvNw8lA7G3Ydska2nkZF|10|20-917-711-4011|270.95|HOUSEHOLD|ly regular accounts hang bold, silent packages. unusual foxes haggle slyly above the special, final depo|
+42|Customer#000000042|ziSrvyyBke|5|15-416-330-4175|8727.01|BUILDING|ssly according to the pinto beans: carefully special requests across the even, pending accounts wake special|
+43|Customer#000000043|ouSbjHk8lh5fKX3zGso3ZSIj9Aa3PoaFd|19|29-316-665-2897|9904.28|MACHINERY|ial requests: carefully pending foxes detect quickly. carefully final courts cajole quickly. carefully|
+44|Customer#000000044|Oi,dOSPwDu4jo4x,,P85E0dmhZGvNtBwi|16|26-190-260-5375|7315.94|AUTOMOBILE|r requests around the unusual, bold a|
+45|Customer#000000045|4v3OcpFgoOmMG,CbnF,4mdC|9|19-715-298-9917|9983.38|AUTOMOBILE|nto beans haggle slyly alongside of t|
+46|Customer#000000046|eaTXWWm10L9|6|16-357-681-2007|5744.59|AUTOMOBILE|ctions. accounts sleep furiously even requests. regular, regular accounts cajole blithely around the final pa|
+47|Customer#000000047|b0UgocSqEW5 gdVbhNT|2|12-427-271-9466|274.58|BUILDING|ions. express, ironic instructions sleep furiously ironic ideas. furi|
+48|Customer#000000048|0UU iPhBupFvemNB|0|10-508-348-5882|3792.50|BUILDING|re fluffily pending foxes. pending, bold platelets sleep slyly. even platelets cajo|
+49|Customer#000000049|cNgAeX7Fqrdf7HQN9EwjUa4nxT,68L FKAxzl|10|20-908-631-4424|4573.94|FURNITURE|nusual foxes! fluffily pending packages maintain to the regular |
+4|Customer#000000004|XxVSJsLAGtn|4|14-128-190-5944|2866.83|MACHINERY| requests. final, regular ideas sleep final accou|
+50|Customer#000000050|9SzDYlkzxByyJ1QeTI o|6|16-658-112-3221|4266.13|MACHINERY|ts. furiously ironic accounts cajole furiously slyly ironic dinos.|
+51|Customer#000000051|uR,wEaiTvo4|12|22-344-885-4251|855.87|FURNITURE|eposits. furiously regular requests integrate carefully packages. furious|
+52|Customer#000000052|7 QOqGqqSy9jfV51BC71jcHJSD0|11|21-186-284-5998|5630.28|HOUSEHOLD|ic platelets use evenly even accounts. stealthy theodolites cajole furiou|
+53|Customer#000000053|HnaxHzTfFTZs8MuCpJyTbZ47Cm4wFOOgib|15|25-168-852-5363|4113.64|HOUSEHOLD|ar accounts are. even foxes are blithely. fluffily pending deposits boost|
+54|Customer#000000054|,k4vf 5vECGWFy,hosTE,|4|14-776-370-4745|868.90|AUTOMOBILE|sual, silent accounts. furiously express accounts cajole special deposits. final, final accounts use furi|
+55|Customer#000000055|zIRBR4KNEl HzaiV3a i9n6elrxzDEh8r8pDom|10|20-180-440-8525|4572.11|MACHINERY|ully unusual packages wake bravely bold packages. unusual requests boost deposits! blithely ironic packages ab|
+56|Customer#000000056|BJYZYJQk4yD5B|10|20-895-685-6920|6530.86|FURNITURE|. notornis wake carefully. carefully fluffy requests are furiously even accounts. slyly expre|
+57|Customer#000000057|97XYbsuOPRXPWU|21|31-835-306-1650|4151.93|AUTOMOBILE|ove the carefully special packages. even, unusual deposits sleep slyly pend|
+58|Customer#000000058|g9ap7Dk1Sv9fcXEWjpMYpBZIRUohi T|13|23-244-493-2508|6478.46|HOUSEHOLD|ideas. ironic ideas affix furiously express, final instructions. regular excuses use quickly e|
+59|Customer#000000059|zLOCP0wh92OtBihgspOGl4|1|11-355-584-3112|3458.60|MACHINERY|ously final packages haggle blithely after the express deposits. furiou|
+5|Customer#000000005|KvpyuHCplrB84WgAiGV6sYpZq7Tj|3|13-750-942-6364|794.47|HOUSEHOLD|n accounts will have to unwind. foxes cajole accor|
+60|Customer#000000060|FyodhjwMChsZmUz7Jz0H|12|22-480-575-5866|2741.87|MACHINERY|latelets. blithely unusual courts boost furiously about the packages. blithely final instruct|
+61|Customer#000000061|9kndve4EAJxhg3veF BfXr7AqOsT39o gtqjaYE|17|27-626-559-8599|1536.24|FURNITURE|egular packages shall have to impress along the |
+62|Customer#000000062|upJK2Dnw13,|7|17-361-978-7059|595.61|MACHINERY|kly special dolphins. pinto beans are slyly. quickly regular accounts are furiously a|
+63|Customer#000000063|IXRSpVWWZraKII|21|31-952-552-9584|9331.13|AUTOMOBILE|ithely even accounts detect slyly above the fluffily ir|
+64|Customer#000000064|MbCeGY20kaKK3oalJD,OT|3|13-558-731-7204|-646.64|BUILDING|structions after the quietly ironic theodolites cajole be|
+65|Customer#000000065|RGT yzQ0y4l0H90P783LG4U95bXQFDRXbWa1sl,X|23|33-733-623-5267|8795.16|AUTOMOBILE|y final foxes serve carefully. theodolites are carefully. pending i|
+66|Customer#000000066|XbsEqXH1ETbJYYtA1A|22|32-213-373-5094|242.77|HOUSEHOLD|le slyly accounts. carefully silent packages benea|
+67|Customer#000000067|rfG0cOgtr5W8 xILkwp9fpCS8|9|19-403-114-4356|8166.59|MACHINERY|indle furiously final, even theodo|
+68|Customer#000000068|o8AibcCRkXvQFh8hF,7o|12|22-918-832-2411|6853.37|HOUSEHOLD| pending pinto beans impress realms. final dependencies |
+69|Customer#000000069|Ltx17nO9Wwhtdbe9QZVxNgP98V7xW97uvSH1prEw|9|19-225-978-5670|1709.28|HOUSEHOLD|thely final ideas around the quickly final dependencies affix carefully quickly final theodolites. final accounts c|
+6|Customer#000000006|sKZz0CsnMD7mp4Xd0YrBvx,LREYKUWAh yVn|20|30-114-968-4951|7638.57|AUTOMOBILE|tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious|
+70|Customer#000000070|mFowIuhnHjp2GjCiYYavkW kUwOjIaTCQ|22|32-828-107-2832|4867.52|FURNITURE|fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be|
+71|Customer#000000071|TlGalgdXWBmMV,6agLyWYDyIz9MKzcY8gl,w6t1B|7|17-710-812-5403|-611.19|HOUSEHOLD|g courts across the regular, final pinto beans are blithely pending ac|
+72|Customer#000000072|putjlmskxE,zs,HqeIA9Wqu7dhgH5BVCwDwHHcf|2|12-759-144-9689|-362.86|FURNITURE|ithely final foxes sleep always quickly bold accounts. final wat|
+73|Customer#000000073|8IhIxreu4Ug6tt5mog4|0|10-473-439-3214|4288.50|BUILDING|usual, unusual packages sleep busily along the furiou|
+74|Customer#000000074|IkJHCA3ZThF7qL7VKcrU nRLl,kylf |4|14-199-862-7209|2764.43|MACHINERY|onic accounts. blithely slow packages would haggle carefully. qui|
+75|Customer#000000075|Dh 6jZ,cwxWLKQfRKkiGrzv6pm|18|28-247-803-9025|6684.10|AUTOMOBILE| instructions cajole even, even deposits. finally bold deposits use above the even pains. slyl|
+76|Customer#000000076|m3sbCvjMOHyaOofH,e UkGPtqc4|0|10-349-718-3044|5745.33|FURNITURE|pecial deposits. ironic ideas boost blithely according to the closely ironic theodolites! furiously final deposits n|
+77|Customer#000000077|4tAE5KdMFGD4byHtXF92vx|17|27-269-357-4674|1738.87|BUILDING|uffily silent requests. carefully ironic asymptotes among the ironic hockey players are carefully bli|
+78|Customer#000000078|HBOta,ZNqpg3U2cSL0kbrftkPwzX|9|19-960-700-9191|7136.97|FURNITURE|ests. blithely bold pinto beans h|
+79|Customer#000000079|n5hH2ftkVRwW8idtD,BmM2|15|25-147-850-4166|5121.28|MACHINERY|es. packages haggle furiously. regular, special requests poach after the quickly express ideas. blithely pending re|
+7|Customer#000000007|TcGe5gaZNgVePxU5kRrvXBfkasDTea|18|28-190-982-9759|9561.95|AUTOMOBILE|ainst the ironic, express theodolites. express, even pinto beans among the exp|
+80|Customer#000000080|K,vtXp8qYB |0|10-267-172-7101|7383.53|FURNITURE|tect among the dependencies. bold accounts engage closely even pinto beans. ca|
+81|Customer#000000081|SH6lPA7JiiNC6dNTrR|20|30-165-277-3269|2023.71|BUILDING|r packages. fluffily ironic requests cajole fluffily. ironically regular theodolit|
+82|Customer#000000082|zhG3EZbap4c992Gj3bK,3Ne,Xn|18|28-159-442-5305|9468.34|AUTOMOBILE|s wake. bravely regular accounts are furiously. regula|
+83|Customer#000000083|HnhTNB5xpnSF20JBH4Ycs6psVnkC3RDf|22|32-817-154-4122|6463.51|BUILDING|ccording to the quickly bold warhorses. final, regular foxes integrate carefully. bold packages nag blithely ev|
+84|Customer#000000084|lpXz6Fwr9945rnbtMc8PlueilS1WmASr CB|11|21-546-818-3802|5174.71|FURNITURE|ly blithe foxes. special asymptotes haggle blithely against the furiously regular depo|
+85|Customer#000000085|siRerlDwiolhYR 8FgksoezycLj|5|15-745-585-8219|3386.64|FURNITURE|ronic ideas use above the slowly pendin|
+86|Customer#000000086|US6EGGHXbTTXPL9SBsxQJsuvy|0|10-677-951-2353|3306.32|HOUSEHOLD|quests. pending dugouts are carefully aroun|
+87|Customer#000000087|hgGhHVSWQl 6jZ6Ev|23|33-869-884-7053|6327.54|FURNITURE|hely ironic requests integrate according to the ironic accounts. slyly regular pla|
+88|Customer#000000088|wtkjBN9eyrFuENSMmMFlJ3e7jE5KXcg|16|26-516-273-2566|8031.44|AUTOMOBILE|s are quickly above the quickly ironic instructions; even requests about the carefully final deposi|
+89|Customer#000000089|dtR, y9JQWUO6FoJExyp8whOU|14|24-394-451-5404|1530.76|FURNITURE|counts are slyly beyond the slyly final accounts. quickly final ideas wake. r|
+8|Customer#000000008|I0B10bB0AymmC, 0PrRYBCP1yGJ8xcBPmWhl5|17|27-147-574-9335|6819.74|BUILDING|among the slyly regular theodolites kindle blithely courts. carefully even theodolites haggle slyly along the ide|
+90|Customer#000000090|QxCzH7VxxYUWwfL7|16|26-603-491-1238|7354.23|BUILDING|sly across the furiously even |
+91|Customer#000000091|S8OMYFrpHwoNHaGBeuS6E 6zhHGZiprw1b7 q|8|18-239-400-3677|4643.14|AUTOMOBILE|onic accounts. fluffily silent pinto beans boost blithely according to the fluffily exp|
+92|Customer#000000092|obP PULk2LH LqNF,K9hcbNqnLAkJVsl5xqSrY,|2|12-446-416-8471|1182.91|MACHINERY|. pinto beans hang slyly final deposits. ac|
+93|Customer#000000093|EHXBr2QGdh|7|17-359-388-5266|2182.52|MACHINERY|press deposits. carefully regular platelets r|
+94|Customer#000000094|IfVNIN9KtkScJ9dUjK3Pg5gY1aFeaXewwf|9|19-953-499-8833|5500.11|HOUSEHOLD|latelets across the bold, final requests sleep according to the fluffily bold accounts. unusual deposits amon|
+95|Customer#000000095|EU0xvmWvOmUUn5J,2z85DQyG7QCJ9Xq7|15|25-923-255-2929|5327.38|MACHINERY|ithely. ruthlessly final requests wake slyly alongside of the furiously silent pinto beans. even the|
+96|Customer#000000096|vWLOrmXhRR|8|18-422-845-1202|6323.92|AUTOMOBILE|press requests believe furiously. carefully final instructions snooze carefully. |
+97|Customer#000000097|OApyejbhJG,0Iw3j rd1M|17|27-588-919-5638|2164.48|AUTOMOBILE|haggle slyly. bold, special ideas are blithely above the thinly bold theo|
+98|Customer#000000098|7yiheXNSpuEAwbswDW|12|22-885-845-6889|-551.37|BUILDING|ages. furiously pending accounts are quickly carefully final foxes: busily pe|
+99|Customer#000000099|szsrOiPtCHVS97Lt|15|25-515-237-9232|4088.65|HOUSEHOLD|cajole slyly about the regular theodolites! furiously bold requests nag along the pending, regular packages. somas|
+9|Customer#000000009|xKiAFTjUsCuxfeleNqefumTrjS|8|18-338-906-3675|8324.07|FURNITURE|r theodolites according to the requests wake thinly excuses: pending requests haggle furiousl|
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/core-site.xml b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/core-site.xml
new file mode 100644
index 0000000..47dfac5
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/core-site.xml
@@ -0,0 +1,18 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+
+<property>
+    <name>fs.default.name</name>
+    <value>hdfs://127.0.0.1:31888</value>
+</property>
+<property>
+    <name>hadoop.tmp.dir</name>
+    <value>/tmp/hadoop</value>
+</property>
+
+
+</configuration>
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/hdfs-site.xml b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/hdfs-site.xml
new file mode 100644
index 0000000..8d29b1d
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/hdfs-site.xml
@@ -0,0 +1,18 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+
+<property>
+   <name>dfs.replication</name>
+   <value>1</value>
+</property>
+
+<property>
+	<name>dfs.block.size</name>
+	<value>65536</value>
+</property>
+
+</configuration>
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/log4j.properties b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/log4j.properties
new file mode 100755
index 0000000..d5e6004
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/log4j.properties
@@ -0,0 +1,94 @@
+# Define some default values that can be overridden by system properties
+hadoop.root.logger=FATAL,console
+hadoop.log.dir=.
+hadoop.log.file=hadoop.log
+
+# Define the root logger to the system property "hadoop.root.logger".
+log4j.rootLogger=${hadoop.root.logger}, EventCounter
+
+# Logging Threshold
+log4j.threshhold=FATAL
+
+#
+# Daily Rolling File Appender
+#
+
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+# 30-day backup
+#log4j.appender.DRFA.MaxBackupIndex=30
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+# Debugging Pattern format
+#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this 
+#
+
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
+
+#
+# TaskLog Appender
+#
+
+#Default values
+hadoop.tasklog.taskid=null
+hadoop.tasklog.noKeepSplits=4
+hadoop.tasklog.totalLogFileSize=100
+hadoop.tasklog.purgeLogSplits=true
+hadoop.tasklog.logsRetainHours=12
+
+log4j.appender.TLA=org.apache.hadoop.mapred.TaskLogAppender
+log4j.appender.TLA.taskId=${hadoop.tasklog.taskid}
+log4j.appender.TLA.totalLogFileSize=${hadoop.tasklog.totalLogFileSize}
+
+log4j.appender.TLA.layout=org.apache.log4j.PatternLayout
+log4j.appender.TLA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+
+#
+# Rolling File Appender
+#
+
+#log4j.appender.RFA=org.apache.log4j.RollingFileAppender
+#log4j.appender.RFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+# Logfile size and and 30-day backups
+#log4j.appender.RFA.MaxFileSize=1MB
+#log4j.appender.RFA.MaxBackupIndex=30
+
+#log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} - %m%n
+#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+#
+# FSNamesystem Audit logging
+# All audit events are logged at INFO level
+#
+log4j.logger.org.apache.hadoop.fs.FSNamesystem.audit=WARN
+
+# Custom Logging levels
+
+#log4j.logger.org.apache.hadoop.mapred.JobTracker=DEBUG
+#log4j.logger.org.apache.hadoop.mapred.TaskTracker=DEBUG
+#log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
+
+# Jets3t library
+log4j.logger.org.jets3t.service.impl.rest.httpclient.RestS3Service=ERROR
+
+#
+# Event Counter Appender
+# Sends counts of logging messages at different severity levels to Hadoop Metrics.
+#
+log4j.appender.EventCounter=org.apache.hadoop.metrics.jvm.EventCounter
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/mapred-site.xml b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/mapred-site.xml
new file mode 100644
index 0000000..39b6505
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/mapred-site.xml
@@ -0,0 +1,25 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+
+  <property>
+    <name>mapred.job.tracker</name>
+    <value>localhost:29007</value>
+  </property>
+  <property>
+     <name>mapred.tasktracker.map.tasks.maximum</name>
+     <value>20</value>
+  </property>
+   <property>
+      <name>mapred.tasktracker.reduce.tasks.maximum</name>
+      <value>20</value>
+   </property>
+   <property>
+      <name>mapred.max.split.size</name>
+      <value>2048</value>
+   </property>
+
+</configuration>
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/topology.xml b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/topology.xml
new file mode 100644
index 0000000..3a0ac7e
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/topology.xml
@@ -0,0 +1,32 @@
+<cluster-topology>
+	<network-switch name="all">
+		<network-switch name="rack1">
+			<terminal name="10.0.0.1" />
+			<terminal name="10.0.0.5" />
+			<terminal name="10.0.0.9" />
+			<terminal name="10.0.0.13" />
+			<terminal name="10.0.0.17" />
+		</network-switch>
+		<network-switch name="rack2">
+			<terminal name="10.0.0.2" />
+			<terminal name="10.0.0.6" />
+			<terminal name="10.0.0.10" />
+			<terminal name="10.0.0.14" />
+			<terminal name="10.0.0.18" />
+		</network-switch>
+		<network-switch name="rack3">
+			<terminal name="10.0.0.3" />
+			<terminal name="10.0.0.7" />
+			<terminal name="10.0.0.11" />
+			<terminal name="10.0.0.15" />
+			<terminal name="10.0.0.19" />
+		</network-switch>
+		<network-switch name="rack4">
+			<terminal name="10.0.0.4" />
+			<terminal name="10.0.0.8" />
+			<terminal name="10.0.0.12" />
+			<terminal name="10.0.0.16" />
+			<terminal name="10.0.0.20" />
+		</network-switch>
+	</network-switch>
+</cluster-topology>
\ No newline at end of file
diff --git a/hyracks/hyracks-hdfs/pom.xml b/hyracks/hyracks-hdfs/pom.xml
new file mode 100644
index 0000000..5ed76e9
--- /dev/null
+++ b/hyracks/hyracks-hdfs/pom.xml
@@ -0,0 +1,19 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<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>
+  <artifactId>hyracks-hdfs</artifactId>
+  <packaging>pom</packaging>
+  <name>hyracks-hdfs</name>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+  </parent>
+
+  <modules>
+    <module>hyracks-hdfs-0.20.2</module>
+    <module>hyracks-hdfs-0.23.1</module>
+    <module>hyracks-hdfs-core</module>
+  </modules>
+</project>
diff --git a/hyracks/hyracks-ipc/pom.xml b/hyracks/hyracks-ipc/pom.xml
index cb0de08..6f5e09f 100644
--- a/hyracks/hyracks-ipc/pom.xml
+++ b/hyracks/hyracks-ipc/pom.xml
@@ -15,8 +15,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml b/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml
index 50c05da..a8fc29e 100644
--- a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml
+++ b/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml
@@ -17,8 +17,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java b/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java
index 47de024..fc06a68 100644
--- a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java
+++ b/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java
@@ -55,6 +55,7 @@
         cmdLineBuffer.append(" -data-ip-address ").append(dataIpAddress);
         cmdLineBuffer.append(" -node-id ").append(nodeId);
         cmdLineBuffer.append(" -cluster-net-ip-address 127.0.0.1");
+        cmdLineBuffer.append(" -result-ip-address 127.0.0.1");
         if (ccPort != 0) {
             cmdLineBuffer.append(" -cc-port ").append(ccPort);
         }
diff --git a/hyracks/hyracks-net/pom.xml b/hyracks/hyracks-net/pom.xml
index a079306..fb486df 100644
--- a/hyracks/hyracks-net/pom.xml
+++ b/hyracks/hyracks-net/pom.xml
@@ -15,8 +15,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java
index dabc8a4..8fa99be 100644
--- a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java
+++ b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java
@@ -58,6 +58,7 @@
 
     ChannelControlBlock allocateChannel() throws NetException {
         synchronized (mConn) {
+       	    cleanupClosedChannels();
             int idx = allocationBitmap.nextClearBit(0);
             if (idx < 0 || idx >= ccbArray.length) {
                 cleanupClosedChannels();
@@ -231,4 +232,4 @@
             ccbArray = Arrays.copyOf(ccbArray, ccbArray.length * 2);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
index c719bc4..e4df6b9 100644
--- a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
+++ b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
@@ -48,9 +48,9 @@
      * Constructor.
      * 
      * @param localAddress
-     *            - TCP/IP socket address to listen on
+     *            - TCP/IP socket address to listen on. Null for non-listening unidirectional sockets
      * @param listener
-     *            - Callback interface to report channel events
+     *            - Callback interface to report channel events. Null for non-listening unidirectional sockets
      * @param nThreads
      *            - Number of threads to use for data transfer
      * @param maxConnectionAttempts
diff --git a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
index d13a17e..a9061e1 100644
--- a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
+++ b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
@@ -45,15 +45,23 @@
     }
 
     public void start(InetSocketAddress localAddress) throws IOException {
-        serverSocketChannel = ServerSocketChannel.open();
-        ServerSocket serverSocket = serverSocketChannel.socket();
-        serverSocket.bind(localAddress);
-        this.localAddress = (InetSocketAddress) serverSocket.getLocalSocketAddress();
+        // Setup a server socket listening channel only if the TCPEndpoint is a listening endpoint.
+        if (localAddress != null) {
+            serverSocketChannel = ServerSocketChannel.open();
+            ServerSocket serverSocket = serverSocketChannel.socket();
+            serverSocket.bind(localAddress);
+            this.localAddress = (InetSocketAddress) serverSocket.getLocalSocketAddress();
+        }
+
         ioThreads = new IOThread[nThreads];
         for (int i = 0; i < ioThreads.length; ++i) {
             ioThreads[i] = new IOThread();
         }
-        ioThreads[0].registerServerSocket(serverSocketChannel);
+
+        if (localAddress != null) {
+            ioThreads[0].registerServerSocket(serverSocketChannel);
+        }
+
         for (int i = 0; i < ioThreads.length; ++i) {
             ioThreads[i].start();
         }
diff --git a/hyracks/hyracks-server/pom.xml b/hyracks/hyracks-server/pom.xml
index 2699c27..e0fc40a 100644
--- a/hyracks/hyracks-server/pom.xml
+++ b/hyracks/hyracks-server/pom.xml
@@ -15,13 +15,15 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
             <configuration>
diff --git a/hyracks/hyracks-storage-am-btree/pom.xml b/hyracks/hyracks-storage-am-btree/pom.xml
index 118d46d..f251d51 100644
--- a/hyracks/hyracks-storage-am-btree/pom.xml
+++ b/hyracks/hyracks-storage-am-btree/pom.xml
@@ -18,8 +18,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-storage-am-common/pom.xml b/hyracks/hyracks-storage-am-common/pom.xml
index 0b32733..dbc4f41d 100644
--- a/hyracks/hyracks-storage-am-common/pom.xml
+++ b/hyracks/hyracks-storage-am-common/pom.xml
@@ -18,8 +18,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-storage-am-invertedindex/pom.xml b/hyracks/hyracks-storage-am-invertedindex/pom.xml
index a647e9d..5fe2d96 100644
--- a/hyracks/hyracks-storage-am-invertedindex/pom.xml
+++ b/hyracks/hyracks-storage-am-invertedindex/pom.xml
@@ -18,8 +18,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index d00bea6..6744f70 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -15,13 +15,13 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
 
-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.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -41,7 +41,7 @@
 
     private FrameTupleAccessor accessor;
     private ArrayTupleBuilder builder;
-    private DataOutput builderDos;
+    private GrowableArray builderFieldData;
     private FrameTupleAppender appender;
     private ByteBuffer writeBuffer;
 
@@ -60,7 +60,7 @@
         accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
         writeBuffer = ctx.allocateFrame();
         builder = new ArrayTupleBuilder(outputRecDesc.getFieldCount());
-        builderDos = builder.getDataOutput();
+        builderFieldData = builder.getFieldData();
         appender = new FrameTupleAppender(ctx.getFrameSize());
         appender.reset(writeBuffer, true);
         writer.open();
@@ -87,7 +87,7 @@
                     builder.reset();
                     try {
                         IToken token = tokenizer.getToken();
-                        token.serializeToken(builderDos);
+                        token.serializeToken(builderFieldData);
                         builder.addFieldEndOffset();
                     } catch (IOException e) {
                         throw new HyracksDataException(e.getMessage());
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
index 99e76dc..3525fc3 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
@@ -202,7 +202,9 @@
     public void endBulkLoad(IIndexBulkLoadContext ictx) throws HyracksDataException {
         // Create entry in btree for last inverted list.
         InvertedIndexBulkLoadContext ctx = (InvertedIndexBulkLoadContext) ictx;
-        createAndInsertBTreeTuple(ctx);
+        if (ctx.lastTuple.getFieldData(0) != null) {
+            createAndInsertBTreeTuple(ctx);
+        }
         btree.endBulkLoad(ctx.btreeBulkLoadCtx);
         ctx.deinit();
     }
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
index 1f886a2..af5dad3 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
@@ -15,7 +15,6 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.impls;
 
-import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -28,6 +27,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -75,7 +75,7 @@
     protected RecordDescriptor queryTokenRecDesc = new RecordDescriptor(
             new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
     protected ArrayTupleBuilder queryTokenBuilder = new ArrayTupleBuilder(queryTokenRecDesc.getFieldCount());
-    protected DataOutput queryTokenDos = queryTokenBuilder.getDataOutput();
+    protected GrowableArray queryTokenFieldData = queryTokenBuilder.getFieldData();
     protected FrameTupleAppender queryTokenAppender;
     protected ByteBuffer queryTokenFrame;
 
@@ -158,7 +158,7 @@
             queryTokenBuilder.reset();
             try {
                 IToken token = queryTokenizer.getToken();
-                token.serializeToken(queryTokenDos);
+                token.serializeToken(queryTokenFieldData);
                 queryTokenBuilder.addFieldEndOffset();
                 // WARNING: assuming one frame is big enough to hold all tokens
                 queryTokenAppender.append(queryTokenBuilder.getFieldEndOffsets(), queryTokenBuilder.getByteArray(), 0,
@@ -294,7 +294,8 @@
         boolean advanceCursor = true;
         boolean advancePrevResult = false;
         int resultTidx = 0;
-
+        currentNumResults = 0;
+        
         resultFrameTupleAcc.reset(prevCurrentBuffer);
         resultFrameTupleApp.reset(newCurrentBuffer, true);
 
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
index 65afa65..2f60952 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
@@ -22,6 +22,7 @@
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 
 public abstract class AbstractUTF8Token implements IToken {
     public static final int GOLDEN_RATIO_32 = 0x09e3779b9;
@@ -97,8 +98,8 @@
     }
 
     @Override
-    public void serializeTokenCount(DataOutput dos) throws IOException {
-        handleCountTypeTag(dos);
-        dos.writeInt(tokenCount);
+    public void serializeTokenCount(GrowableArray out) throws IOException {
+        handleCountTypeTag(out.getDataOutput());
+        out.getDataOutput().writeInt(tokenCount);
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
index b7bb828..a1a4354 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
@@ -19,10 +19,10 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import java.io.DataOutput;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 
 public class HashedUTF8NGramToken extends UTF8NGramToken {
     public HashedUTF8NGramToken(byte tokenTypeTag, byte countTypeTag) {
@@ -30,8 +30,8 @@
     }
 
     @Override
-    public void serializeToken(DataOutput dos) throws IOException {
-        handleTokenTypeTag(dos);
+    public void serializeToken(GrowableArray out) throws IOException {
+        handleTokenTypeTag(out.getDataOutput());
 
         int hash = GOLDEN_RATIO_32;
 
@@ -59,6 +59,6 @@
         // token count
         hash += tokenCount;
 
-        dos.writeInt(hash);
+        out.getDataOutput().writeInt(hash);
     }
 }
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
index 42ed053..20405c6 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
@@ -19,10 +19,10 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import java.io.DataOutput;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 
 public class HashedUTF8WordToken extends UTF8WordToken {
 
@@ -76,12 +76,12 @@
     }
 
     @Override
-    public void serializeToken(DataOutput dos) throws IOException {
+    public void serializeToken(GrowableArray out) throws IOException {
         if (tokenTypeTag > 0) {
-            dos.write(tokenTypeTag);
+            out.getDataOutput().write(tokenTypeTag);
         }
 
         // serialize hash value
-        dos.writeInt(hash);
+        out.getDataOutput().writeInt(hash);
     }
 }
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java
index c1840d7..47467a1 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java
@@ -19,9 +19,10 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import java.io.DataOutput;
 import java.io.IOException;
 
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+
 public interface IToken {
 	public byte[] getData();
 
@@ -34,7 +35,7 @@
 	public void reset(byte[] data, int start, int length, int tokenLength,
 			int tokenCount);
 
-	public void serializeToken(DataOutput dos) throws IOException;
+	public void serializeToken(GrowableArray out) throws IOException;
 
-	public void serializeTokenCount(DataOutput dos) throws IOException;
+	public void serializeTokenCount(GrowableArray out) throws IOException;
 }
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
index 59cadc8..8cb9818 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
@@ -19,10 +19,10 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import java.io.DataOutput;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
 
 public class UTF8NGramToken extends AbstractUTF8Token implements INGramToken {
@@ -49,34 +49,39 @@
     }
 
     @Override
-    public void serializeToken(DataOutput dos) throws IOException {
-        handleTokenTypeTag(dos);
+    public void serializeToken(GrowableArray out) throws IOException {
+        handleTokenTypeTag(out.getDataOutput());
+        int tokenUTF8LenOff = out.getLength();
 
         // regular chars
         int numRegChars = tokenLength - numPreChars - numPostChars;
 
         // assuming pre and post char need 1-byte each in utf8
-        int tokenUTF8Len = getLowerCaseUTF8Len(numRegChars) + numPreChars + numPostChars;
+        int tokenUTF8Len = numPreChars + numPostChars;
 
-        // write utf8 length indicator
-        StringUtils.writeUTF8Len(tokenUTF8Len, dos);
+        // Write dummy UTF length which will be correctly set later.
+        out.getDataOutput().writeShort(0);
 
         // pre chars
         for (int i = 0; i < numPreChars; i++) {
-            StringUtils.writeCharAsModifiedUTF8(PRECHAR, dos);
+            StringUtils.writeCharAsModifiedUTF8(PRECHAR, out.getDataOutput());
         }
 
         int pos = start;
         for (int i = 0; i < numRegChars; i++) {
             char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
-            StringUtils.writeCharAsModifiedUTF8(c, dos);
+            tokenUTF8Len += StringUtils.writeCharAsModifiedUTF8(c, out.getDataOutput());
             pos += UTF8StringPointable.charSize(data, pos);
         }
 
         // post chars
         for (int i = 0; i < numPostChars; i++) {
-            StringUtils.writeCharAsModifiedUTF8(POSTCHAR, dos);
+            StringUtils.writeCharAsModifiedUTF8(POSTCHAR, out.getDataOutput());
         }
+
+        // Set UTF length of token.
+        out.getByteArray()[tokenUTF8LenOff] = (byte) ((tokenUTF8Len >>> 8) & 0xFF);
+        out.getByteArray()[tokenUTF8LenOff + 1] = (byte) ((tokenUTF8Len >>> 0) & 0xFF);
     }
 
     public void setNumPrePostChars(int numPreChars, int numPostChars) {
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
index 97a1e12..9d7fe7c 100644
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
+++ b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
@@ -19,10 +19,10 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
 
-import java.io.DataOutput;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
 
 public class UTF8WordToken extends AbstractUTF8Token {
@@ -32,16 +32,20 @@
     }
 
     @Override
-    public void serializeToken(DataOutput dos) throws IOException {
-        handleTokenTypeTag(dos);
-
-        int tokenUTF8Len = getLowerCaseUTF8Len(tokenLength);
-        StringUtils.writeUTF8Len(tokenUTF8Len, dos);
+    public void serializeToken(GrowableArray out) throws IOException {
+        handleTokenTypeTag(out.getDataOutput());
+        int tokenUTF8LenOff = out.getLength();
+        int tokenUTF8Len = 0;
+        // Write dummy UTF length which will be correctly set later.
+        out.getDataOutput().writeShort(0);
         int pos = start;
         for (int i = 0; i < tokenLength; i++) {
             char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
-            StringUtils.writeCharAsModifiedUTF8(c, dos);
+            tokenUTF8Len += StringUtils.writeCharAsModifiedUTF8(c, out.getDataOutput());
             pos += UTF8StringPointable.charSize(data, pos);
         }
+        // Set UTF length of token.
+        out.getByteArray()[tokenUTF8LenOff] = (byte) ((tokenUTF8Len >>> 8) & 0xFF);
+        out.getByteArray()[tokenUTF8LenOff + 1] = (byte) ((tokenUTF8Len >>> 0) & 0xFF);
     }
 }
diff --git a/hyracks/hyracks-storage-am-rtree/pom.xml b/hyracks/hyracks-storage-am-rtree/pom.xml
index 61620ec..6c2d734 100644
--- a/hyracks/hyracks-storage-am-rtree/pom.xml
+++ b/hyracks/hyracks-storage-am-rtree/pom.xml
@@ -18,8 +18,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 9818dce..d9b7b97 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -40,9 +40,10 @@
             IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
-            IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, IOperationCallbackProvider opCallbackProvider) {
+            IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput,
+            IOperationCallbackProvider opCallbackProvider) {
         super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, dataflowHelperFactory, null, false, opCallbackProvider);
+                comparatorFactories, dataflowHelperFactory, null, retainInput, opCallbackProvider);
         this.keyFields = keyFields;
     }
 
diff --git a/hyracks/hyracks-storage-common/pom.xml b/hyracks/hyracks-storage-common/pom.xml
index 289171a..3360097 100644
--- a/hyracks/hyracks-storage-common/pom.xml
+++ b/hyracks/hyracks-storage-common/pom.xml
@@ -18,8 +18,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-test-support/pom.xml b/hyracks/hyracks-test-support/pom.xml
index 25a5378..89233c9 100644
--- a/hyracks/hyracks-test-support/pom.xml
+++ b/hyracks/hyracks-test-support/pom.xml
@@ -18,8 +18,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
index c122b25..0ca93b2 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -101,6 +102,11 @@
     }
 
     @Override
+    public IDatasetPartitionManager getDatasetPartitionManager() {
+        return null;
+    }
+
+    @Override
     public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception {
         // TODO Auto-generated method stub
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
index 7b03a71..d0bb883 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
@@ -18,8 +18,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
index 2cf6ce2..59c8c46 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
@@ -18,9 +18,10 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
           <encoding>UTF-8</encoding>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java
index 5f15a91..3fb6407 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java
@@ -33,6 +33,7 @@
 import org.junit.Before;
 import org.junit.Test;
 
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.AbstractUTF8Token;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
@@ -41,207 +42,196 @@
 
 public class NGramTokenizerTest {
 
-	private char PRECHAR = '#';
-	private char POSTCHAR = '$';
+    private char PRECHAR = '#';
+    private char POSTCHAR = '$';
 
-	private String str = "Jürgen S. Generic's Car";
-	private byte[] inputBuffer;
+    private String str = "Jürgen S. Generic's Car";
+    private byte[] inputBuffer;
 
-	private int gramLength = 3;
+    private int gramLength = 3;
 
-	private void getExpectedGrams(String s, int gramLength,
-			ArrayList<String> grams, boolean prePost) {
+    private void getExpectedGrams(String s, int gramLength, ArrayList<String> grams, boolean prePost) {
 
-		String tmp = s.toLowerCase();
-		if (prePost) {
-			StringBuilder preBuilder = new StringBuilder();
-			for (int i = 0; i < gramLength - 1; i++) {
-				preBuilder.append(PRECHAR);
-			}
-			String pre = preBuilder.toString();
+        String tmp = s.toLowerCase();
+        if (prePost) {
+            StringBuilder preBuilder = new StringBuilder();
+            for (int i = 0; i < gramLength - 1; i++) {
+                preBuilder.append(PRECHAR);
+            }
+            String pre = preBuilder.toString();
 
-			StringBuilder postBuilder = new StringBuilder();
-			for (int i = 0; i < gramLength - 1; i++) {
-				postBuilder.append(POSTCHAR);
-			}
-			String post = postBuilder.toString();
+            StringBuilder postBuilder = new StringBuilder();
+            for (int i = 0; i < gramLength - 1; i++) {
+                postBuilder.append(POSTCHAR);
+            }
+            String post = postBuilder.toString();
 
-			tmp = pre + s.toLowerCase() + post;
-		}
+            tmp = pre + s.toLowerCase() + post;
+        }
 
-		for (int i = 0; i < tmp.length() - gramLength + 1; i++) {
-			String gram = tmp.substring(i, i + gramLength);
-			grams.add(gram);
-		}
-	}
+        for (int i = 0; i < tmp.length() - gramLength + 1; i++) {
+            String gram = tmp.substring(i, i + gramLength);
+            grams.add(gram);
+        }
+    }
 
-	@Before
-	public void init() throws Exception {
-		// serialize string into bytes
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		DataOutput dos = new DataOutputStream(baos);
-		dos.writeUTF(str);
-		inputBuffer = baos.toByteArray();
-	}
+    @Before
+    public void init() throws Exception {
+        // serialize string into bytes
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutput dos = new DataOutputStream(baos);
+        dos.writeUTF(str);
+        inputBuffer = baos.toByteArray();
+    }
 
-	void runTestNGramTokenizerWithCountedHashedUTF8Tokens(boolean prePost)
-			throws IOException {
-		HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
-		NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(
-				gramLength, prePost, false, false, tokenFactory);
-		tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+    void runTestNGramTokenizerWithCountedHashedUTF8Tokens(boolean prePost) throws IOException {
+        HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
+        NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, false,
+                false, tokenFactory);
+        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
 
-		ArrayList<String> expectedGrams = new ArrayList<String>();
-		getExpectedGrams(str, gramLength, expectedGrams, prePost);
-		ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
-		HashMap<String, Integer> gramCounts = new HashMap<String, Integer>();
-		for (String s : expectedGrams) {
-			Integer count = gramCounts.get(s);
-			if (count == null) {
-				count = 1;
-				gramCounts.put(s, count);
-			} else {
-				count++;
-			}
+        ArrayList<String> expectedGrams = new ArrayList<String>();
+        getExpectedGrams(str, gramLength, expectedGrams, prePost);
+        ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
+        HashMap<String, Integer> gramCounts = new HashMap<String, Integer>();
+        for (String s : expectedGrams) {
+            Integer count = gramCounts.get(s);
+            if (count == null) {
+                count = 1;
+                gramCounts.put(s, count);
+            } else {
+                count++;
+            }
 
-			int hash = tokenHash(s, count);
-			expectedHashedGrams.add(hash);
-		}
+            int hash = tokenHash(s, count);
+            expectedHashedGrams.add(hash);
+        }
 
-		int tokenCount = 0;
+        int tokenCount = 0;
 
-		while (tokenizer.hasNext()) {
-			tokenizer.next();
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
 
-			// serialize hashed token
-			ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
-			DataOutput tokenDos = new DataOutputStream(tokenBaos);
+            // serialize hashed token
+            GrowableArray tokenStorage = new GrowableArray();
 
-			IToken token = tokenizer.getToken();
-			token.serializeToken(tokenDos);
+            IToken token = tokenizer.getToken();
+            token.serializeToken(tokenStorage);
 
-			// deserialize token
-			ByteArrayInputStream bais = new ByteArrayInputStream(
-					tokenBaos.toByteArray());
-			DataInput in = new DataInputStream(bais);
+            // deserialize token
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
+            DataInput in = new DataInputStream(bais);
 
-			Integer hashedGram = in.readInt();
+            Integer hashedGram = in.readInt();
 
-			// System.out.println(hashedGram);
+            // System.out.println(hashedGram);
 
-			Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
+            Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
 
-			tokenCount++;
-		}
-		// System.out.println("---------");
-	}
+            tokenCount++;
+        }
+        // System.out.println("---------");
+    }
 
-	void runTestNGramTokenizerWithHashedUTF8Tokens(boolean prePost)
-			throws IOException {
-		HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
-		NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(
-				gramLength, prePost, true, false, tokenFactory);
-		tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+    void runTestNGramTokenizerWithHashedUTF8Tokens(boolean prePost) throws IOException {
+        HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
+        NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, true, false,
+                tokenFactory);
+        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
 
-		ArrayList<String> expectedGrams = new ArrayList<String>();
-		getExpectedGrams(str, gramLength, expectedGrams, prePost);
-		ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
-		for (String s : expectedGrams) {
-			int hash = tokenHash(s, 1);
-			expectedHashedGrams.add(hash);
-		}
+        ArrayList<String> expectedGrams = new ArrayList<String>();
+        getExpectedGrams(str, gramLength, expectedGrams, prePost);
+        ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
+        for (String s : expectedGrams) {
+            int hash = tokenHash(s, 1);
+            expectedHashedGrams.add(hash);
+        }
 
-		int tokenCount = 0;
+        int tokenCount = 0;
 
-		while (tokenizer.hasNext()) {
-			tokenizer.next();
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
 
-			// serialize hashed token
-			ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
-			DataOutput tokenDos = new DataOutputStream(tokenBaos);
+            // serialize hashed token
+            GrowableArray tokenStorage = new GrowableArray();
 
-			IToken token = tokenizer.getToken();
-			token.serializeToken(tokenDos);
+            IToken token = tokenizer.getToken();
+            token.serializeToken(tokenStorage);
 
-			// deserialize token
-			ByteArrayInputStream bais = new ByteArrayInputStream(
-					tokenBaos.toByteArray());
-			DataInput in = new DataInputStream(bais);
+            // deserialize token
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
+            DataInput in = new DataInputStream(bais);
 
-			Integer hashedGram = in.readInt();
+            Integer hashedGram = in.readInt();
 
-			// System.out.println(hashedGram);
+            // System.out.println(hashedGram);
 
-			Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
+            Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
 
-			tokenCount++;
-		}
-		// System.out.println("---------");
-	}
+            tokenCount++;
+        }
+        // System.out.println("---------");
+    }
 
-	void runTestNGramTokenizerWithUTF8Tokens(boolean prePost)
-			throws IOException {
-		UTF8NGramTokenFactory tokenFactory = new UTF8NGramTokenFactory();
-		NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(
-				gramLength, prePost, true, false, tokenFactory);
-		tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+    void runTestNGramTokenizerWithUTF8Tokens(boolean prePost) throws IOException {
+        UTF8NGramTokenFactory tokenFactory = new UTF8NGramTokenFactory();
+        NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, true, false,
+                tokenFactory);
+        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
 
-		ArrayList<String> expectedGrams = new ArrayList<String>();
-		getExpectedGrams(str, gramLength, expectedGrams, prePost);
+        ArrayList<String> expectedGrams = new ArrayList<String>();
+        getExpectedGrams(str, gramLength, expectedGrams, prePost);
 
-		int tokenCount = 0;
+        int tokenCount = 0;
 
-		while (tokenizer.hasNext()) {
-			tokenizer.next();
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
 
-			// serialize hashed token
-			ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
-			DataOutput tokenDos = new DataOutputStream(tokenBaos);
+            // serialize hashed token
+            GrowableArray tokenStorage = new GrowableArray();
 
-			IToken token = tokenizer.getToken();
-			token.serializeToken(tokenDos);
+            IToken token = tokenizer.getToken();
+            token.serializeToken(tokenStorage);
 
-			// deserialize token
-			ByteArrayInputStream bais = new ByteArrayInputStream(
-					tokenBaos.toByteArray());
-			DataInput in = new DataInputStream(bais);
+            // deserialize token
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
+            DataInput in = new DataInputStream(bais);
 
-			String strGram = in.readUTF();
+            String strGram = in.readUTF();
 
-			// System.out.println("\"" + strGram + "\"");
+            // System.out.println("\"" + strGram + "\"");
 
-			Assert.assertEquals(expectedGrams.get(tokenCount), strGram);
+            Assert.assertEquals(expectedGrams.get(tokenCount), strGram);
 
-			tokenCount++;
-		}
-		// System.out.println("---------");
-	}
+            tokenCount++;
+        }
+        // System.out.println("---------");
+    }
 
-	@Test
-	public void testNGramTokenizerWithCountedHashedUTF8Tokens()
-			throws Exception {
-		runTestNGramTokenizerWithCountedHashedUTF8Tokens(false);
-		runTestNGramTokenizerWithCountedHashedUTF8Tokens(true);
-	}
+    @Test
+    public void testNGramTokenizerWithCountedHashedUTF8Tokens() throws Exception {
+        runTestNGramTokenizerWithCountedHashedUTF8Tokens(false);
+        runTestNGramTokenizerWithCountedHashedUTF8Tokens(true);
+    }
 
-	@Test
-	public void testNGramTokenizerWithHashedUTF8Tokens() throws Exception {
-		runTestNGramTokenizerWithHashedUTF8Tokens(false);
-		runTestNGramTokenizerWithHashedUTF8Tokens(true);
-	}
+    @Test
+    public void testNGramTokenizerWithHashedUTF8Tokens() throws Exception {
+        runTestNGramTokenizerWithHashedUTF8Tokens(false);
+        runTestNGramTokenizerWithHashedUTF8Tokens(true);
+    }
 
-	@Test
-	public void testNGramTokenizerWithUTF8Tokens() throws IOException {
-		runTestNGramTokenizerWithUTF8Tokens(false);
-		runTestNGramTokenizerWithUTF8Tokens(true);
-	}
+    @Test
+    public void testNGramTokenizerWithUTF8Tokens() throws IOException {
+        runTestNGramTokenizerWithUTF8Tokens(false);
+        runTestNGramTokenizerWithUTF8Tokens(true);
+    }
 
-	public int tokenHash(String token, int tokenCount) {
-		int h = AbstractUTF8Token.GOLDEN_RATIO_32;
-		for (int i = 0; i < token.length(); i++) {
-			h ^= token.charAt(i);
-			h *= AbstractUTF8Token.GOLDEN_RATIO_32;
-		}
-		return h + tokenCount;
-	}
+    public int tokenHash(String token, int tokenCount) {
+        int h = AbstractUTF8Token.GOLDEN_RATIO_32;
+        for (int i = 0; i < token.length(); i++) {
+            h ^= token.charAt(i);
+            h *= AbstractUTF8Token.GOLDEN_RATIO_32;
+        }
+        return h + tokenCount;
+    }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
index c3c9b99..47a068b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
@@ -27,6 +27,7 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -112,20 +113,19 @@
 	}
 
 	private class TokenIdPair implements Comparable<TokenIdPair> {
-		public ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
-		public DataOutputStream dos = new DataOutputStream(baaos);
+	    public final GrowableArray tokenStorage = new GrowableArray();
 		public int id;
 
 		TokenIdPair(IToken token, int id) throws IOException {
-			token.serializeToken(dos);
+			token.serializeToken(tokenStorage);
 			this.id = id;
 		}
 
 		@Override
 		public int compareTo(TokenIdPair o) {
-			int cmp = btreeBinCmps[0].compare(baaos.getByteArray(), 0,
-					baaos.getByteArray().length, o.baaos.getByteArray(), 0,
-					o.baaos.getByteArray().length);
+			int cmp = btreeBinCmps[0].compare(tokenStorage.getByteArray(), 0,
+			        tokenStorage.getByteArray().length, o.tokenStorage.getByteArray(), 0,
+					o.tokenStorage.getByteArray().length);
 			if (cmp == 0) {
 				return id - o.id;
 			} else {
@@ -157,8 +157,8 @@
 
 		for (TokenIdPair t : pairs) {
 			tb.reset();
-			tb.addField(t.baaos.getByteArray(), 0,
-					t.baaos.getByteArray().length);
+			tb.addField(t.tokenStorage.getByteArray(), 0,
+					t.tokenStorage.getByteArray().length);
 			IntegerSerializerDeserializer.INSTANCE.serialize(t.id, tb.getDataOutput());
 			tb.addFieldEndOffset();
 			tuple.reset(tb.getFieldEndOffsets(), tb.getByteArray());
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java
index 53fb96d..810c5f5 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java
@@ -34,6 +34,7 @@
 import org.junit.Before;
 import org.junit.Test;
 
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.AbstractUTF8Token;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
@@ -127,14 +128,13 @@
             tokenizer.next();
 
             // serialize token
-            ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
-            DataOutput tokenDos = new DataOutputStream(tokenBaos);
+            GrowableArray tokenStorage = new GrowableArray();
 
             IToken token = tokenizer.getToken();
-            token.serializeToken(tokenDos);
+            token.serializeToken(tokenStorage);
 
             // deserialize token
-            ByteArrayInputStream bais = new ByteArrayInputStream(tokenBaos.toByteArray());
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
             DataInput in = new DataInputStream(bais);
 
             Integer hashedToken = in.readInt();
@@ -159,14 +159,13 @@
             tokenizer.next();
 
             // serialize token
-            ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
-            DataOutput tokenDos = new DataOutputStream(tokenBaos);
+            GrowableArray tokenStorage = new GrowableArray();
 
             IToken token = tokenizer.getToken();
-            token.serializeToken(tokenDos);
+            token.serializeToken(tokenStorage);
 
             // deserialize token
-            ByteArrayInputStream bais = new ByteArrayInputStream(tokenBaos.toByteArray());
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
             DataInput in = new DataInputStream(bais);
 
             Integer hashedToken = in.readInt();
@@ -191,14 +190,13 @@
             tokenizer.next();
 
             // serialize hashed token
-            ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
-            DataOutput tokenDos = new DataOutputStream(tokenBaos);
+            GrowableArray tokenStorage = new GrowableArray();
 
             IToken token = tokenizer.getToken();
-            token.serializeToken(tokenDos);
+            token.serializeToken(tokenStorage);
 
             // deserialize token
-            ByteArrayInputStream bais = new ByteArrayInputStream(tokenBaos.toByteArray());
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
             DataInput in = new DataInputStream(bais);
 
             String strToken = in.readUTF();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
index ea86042..7b1a3f3 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
@@ -18,8 +18,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
index bd10e13..8e429f9 100644
--- a/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
+++ b/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
@@ -18,8 +18,9 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
+          <fork>true</fork>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/hyracks-yarn/hyracks-yarn-am/pom.xml b/hyracks/hyracks-yarn/hyracks-yarn-am/pom.xml
index 9e453a6..d33ddc5 100644
--- a/hyracks/hyracks-yarn/hyracks-yarn-am/pom.xml
+++ b/hyracks/hyracks-yarn/hyracks-yarn-am/pom.xml
@@ -14,8 +14,8 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
         </configuration>
       </plugin>
       <plugin>
diff --git a/hyracks/hyracks-yarn/hyracks-yarn-client/pom.xml b/hyracks/hyracks-yarn/hyracks-yarn-client/pom.xml
index 08935a7..649aa6c 100644
--- a/hyracks/hyracks-yarn/hyracks-yarn-client/pom.xml
+++ b/hyracks/hyracks-yarn/hyracks-yarn-client/pom.xml
@@ -14,8 +14,8 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
         </configuration>
       </plugin>
       <plugin>
diff --git a/hyracks/hyracks-yarn/hyracks-yarn-common/pom.xml b/hyracks/hyracks-yarn/hyracks-yarn-common/pom.xml
index 3aaf4a2..fe210fd 100644
--- a/hyracks/hyracks-yarn/hyracks-yarn-common/pom.xml
+++ b/hyracks/hyracks-yarn/hyracks-yarn-common/pom.xml
@@ -14,8 +14,8 @@
         <artifactId>maven-compiler-plugin</artifactId>
         <version>2.0.2</version>
         <configuration>
-          <source>1.6</source>
-          <target>1.6</target>
+          <source>1.7</source>
+          <target>1.7</target>
         </configuration>
       </plugin>
     </plugins>
diff --git a/hyracks/pom.xml b/hyracks/pom.xml
index 570421e..b699542 100644
--- a/hyracks/pom.xml
+++ b/hyracks/pom.xml
@@ -1,4 +1,4 @@
-
+<?xml version="1.0" encoding="UTF-8"?>
 <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>
@@ -8,24 +8,9 @@
   <name>hyracks</name>
 
   <properties>
-    <jvm.extraargs />
+    <jvm.extraargs/>
   </properties>
 
-  <profiles>
-    <profile>
-      <id>macosx</id>
-      <activation>
-        <os>
-          <name>mac os x</name>
-        </os>
-        <jdk>1.7</jdk>
-      </activation>
-      <properties>
-        <jvm.extraargs>-Djava.nio.channels.spi.SelectorProvider=sun.nio.ch.KQueueSelectorProvider</jvm.extraargs>
-      </properties>
-    </profile>
-  </profiles>
-
   <build>
     <plugins>
       <plugin>
@@ -44,6 +29,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
+        <version>2.13</version>
         <configuration>
             <forkMode>pertest</forkMode>
             <argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties -Xdebug -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${jvm.extraargs}</argLine>
@@ -57,6 +43,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-changelog-plugin</artifactId>
+        <version>2.2</version>
       </plugin>
     </plugins>
   </reporting>
@@ -96,6 +83,8 @@
   <modules>
     <module>hyracks-ipc</module>
     <module>hyracks-api</module>
+    <module>hyracks-comm</module>
+    <module>hyracks-client</module>
     <module>hyracks-dataflow-common</module>
     <module>hyracks-dataflow-std</module>
     <module>hyracks-dataflow-hadoop</module>
@@ -116,5 +105,7 @@
     <module>hyracks-hadoop-compat</module>
     <!--module>hyracks-yarn</module-->
     <module>hyracks-maven-plugins</module>
+    <module>hyracks-hdfs</module>
+    <module>hyracks-dist</module>
   </modules>
 </project>