Code move from internal SVN

git-svn-id: https://hyracks.googlecode.com/svn/trunk/hyracks@2 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-core/src/main/assembly/binary-assembly.xml b/hyracks-core/src/main/assembly/binary-assembly.xml
new file mode 100644
index 0000000..cd598d9
--- /dev/null
+++ b/hyracks-core/src/main/assembly/binary-assembly.xml
@@ -0,0 +1,23 @@
+<assembly>
+  <id>binary-assembly</id>
+  <formats>
+    <format>zip</format>
+    <format>dir</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <fileSets>
+    <fileSet>
+      <directory>target/appassembler/bin</directory>
+      <outputDirectory>bin</outputDirectory>
+      <fileMode>0755</fileMode>
+    </fileSet>
+    <fileSet>
+      <directory>target/appassembler/lib</directory>
+      <outputDirectory>lib</outputDirectory>
+    </fileSet>
+    <fileSet>
+      <directory>docs</directory>
+      <outputDirectory>docs</outputDirectory>
+    </fileSet>
+  </fileSets>
+</assembly>
diff --git a/hyracks-core/src/main/assembly/source-assembly.xml b/hyracks-core/src/main/assembly/source-assembly.xml
new file mode 100644
index 0000000..82db140
--- /dev/null
+++ b/hyracks-core/src/main/assembly/source-assembly.xml
@@ -0,0 +1,17 @@
+<assembly>
+  <id>source-assembly</id>
+  <formats>
+    <format>zip</format>
+    <format>dir</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <fileSets>
+    <fileSet>
+      <directory>.</directory>
+      <outputDirectory>hyracks-core</outputDirectory>
+      <excludes>
+        <exclude>target/**</exclude>
+      </excludes>
+    </fileSet>
+  </fileSets>
+</assembly>
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/Endpoint.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/Endpoint.java
new file mode 100644
index 0000000..c0d23c1
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/Endpoint.java
@@ -0,0 +1,65 @@
+/*
+ * 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.comm;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+public final class Endpoint implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final UUID id;
+
+    private final NetworkAddress address;
+
+    private final int receiverIndex;
+
+    public Endpoint(NetworkAddress address, int receiverIndex) throws Exception {
+        id = UUID.randomUUID();
+        this.address = address;
+        this.receiverIndex = receiverIndex;
+    }
+
+    public UUID getEndpointId() {
+        return id;
+    }
+
+    public NetworkAddress getNetworkAddress() {
+        return address;
+    }
+
+    public int getReceiverIndex() {
+        return receiverIndex;
+    }
+
+    @Override
+    public int hashCode() {
+        return id.hashCode() + address.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof Endpoint)) {
+            return false;
+        }
+        Endpoint oe = (Endpoint) o;
+        return oe.id.equals(id) && oe.address.equals(address);
+    }
+
+    @Override
+    public String toString() {
+        return "[" + address + ":" + id + "]";
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/FrameConstants.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/FrameConstants.java
new file mode 100644
index 0000000..cc76bd4
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/FrameConstants.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.comm;
+
+public interface FrameConstants {
+    public static final int SIZE_LEN = 4;
+
+    public static final boolean DEBUG_FRAME_IO = false;
+
+    public static final int FRAME_FIELD_MAGIC = 0x12345678;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionDemultiplexer.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionDemultiplexer.java
new file mode 100644
index 0000000..0587722
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionDemultiplexer.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.comm;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IConnectionDemultiplexer {
+    public int getSenderCount();
+
+    public IConnectionEntry findNextReadyEntry(int lastReadSender);
+
+    public void unreadyEntry(int index);
+
+    public int closeEntry(int index) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionEntry.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionEntry.java
new file mode 100644
index 0000000..125c1aa
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionEntry.java
@@ -0,0 +1,35 @@
+/*
+ * 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.comm;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+
+public interface IConnectionEntry {
+    ByteBuffer getReadBuffer();
+
+    SelectionKey getSelectionKey();
+
+    void setDataReceiveListener(IDataReceiveListener listener);
+
+    void attach(Object attachment);
+
+    Object getAttachment();
+
+    void close() throws IOException;
+
+    void write(ByteBuffer buffer);
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java
new file mode 100644
index 0000000..463e4e1
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.comm;
+
+import java.io.IOException;
+
+public interface IDataReceiveListener {
+    public void dataReceived(IConnectionEntry entry) throws IOException;
+
+    public void eos(IConnectionEntry entry);
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListenerFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListenerFactory.java
new file mode 100644
index 0000000..91f921c
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListenerFactory.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.comm;
+
+import java.util.UUID;
+
+public interface IDataReceiveListenerFactory {
+    public IDataReceiveListener getDataReceiveListener(UUID endpointUUID, IConnectionEntry entry, int senderIndex);
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameReader.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameReader.java
new file mode 100644
index 0000000..0188578
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameReader.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.comm;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrameReader {
+    public void open() throws HyracksDataException;
+
+    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException;
+
+    public void close() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameWriter.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameWriter.java
new file mode 100644
index 0000000..eccd6cb
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameWriter.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.comm;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrameWriter {
+    public void open() throws HyracksDataException;
+
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException;
+
+    public void close() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
new file mode 100644
index 0000000..868221d
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
@@ -0,0 +1,58 @@
+/*
+ * 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.comm;
+
+import java.io.Serializable;
+import java.net.InetAddress;
+
+public final class NetworkAddress implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final InetAddress ipAddress;
+
+    private final int port;
+
+    public NetworkAddress(InetAddress ipAddress, int port) {
+        this.ipAddress = ipAddress;
+        this.port = port;
+    }
+
+    public InetAddress getIpAddress() {
+        return ipAddress;
+    }
+
+    public int getPort() {
+        return port;
+    }
+
+    @Override
+    public String toString() {
+        return ipAddress + ":" + port;
+    }
+
+    @Override
+    public int hashCode() {
+        return ipAddress.hashCode() + port;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof NetworkAddress)) {
+            return false;
+        }
+        NetworkAddress on = (NetworkAddress) o;
+        return on.port == port && on.ipAddress.equals(ipAddress);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/constraints/AbsoluteLocationConstraint.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/constraints/AbsoluteLocationConstraint.java
new file mode 100644
index 0000000..858bb65
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/constraints/AbsoluteLocationConstraint.java
@@ -0,0 +1,45 @@
+/*
+ * 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.constraints;
+
+public class AbsoluteLocationConstraint extends LocationConstraint {
+    private static final long serialVersionUID = 1L;
+
+    private String nodeId;
+
+    public AbsoluteLocationConstraint() {
+    }
+
+    public AbsoluteLocationConstraint(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    @Override
+    public ConstraintType getConstraintType() {
+        return ConstraintType.ABSOLUTE;
+    }
+
+    public String getLocationId() {
+        return nodeId;
+    }
+
+    public void setLocationId(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    public String toString() {
+        return getConstraintType() + ":" + nodeId;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/constraints/LocationConstraint.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/constraints/LocationConstraint.java
new file mode 100644
index 0000000..e88a1b2
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/constraints/LocationConstraint.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.constraints;
+
+import java.io.Serializable;
+
+public abstract class LocationConstraint implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public enum ConstraintType {
+        ABSOLUTE,
+    }
+
+    public abstract ConstraintType getConstraintType();
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraint.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraint.java
new file mode 100644
index 0000000..c825539
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraint.java
@@ -0,0 +1,36 @@
+/*
+ * 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.constraints;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+public class PartitionConstraint implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final LocationConstraint[] locationConstraints;
+
+    public PartitionConstraint(LocationConstraint[] locationConstraints) {
+        this.locationConstraints = locationConstraints;
+    }
+
+    public LocationConstraint[] getLocationConstraints() {
+        return locationConstraints;
+    }
+
+    public String toString() {
+        return Arrays.deepToString(locationConstraints);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/controller/IClusterController.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/controller/IClusterController.java
new file mode 100644
index 0000000..af86093
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/controller/IClusterController.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.api.controller;
+
+import java.rmi.Remote;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.api.job.statistics.JobStatistics;
+import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
+
+public interface IClusterController extends Remote {
+    public void registerNode(INodeController nodeController) throws Exception;
+
+    public void unregisterNode(INodeController nodeController) throws Exception;
+
+    public INodeController lookupNode(String id) throws Exception;
+
+    public void notifyStageletComplete(UUID jobId, UUID stageId, String nodeId, StageletStatistics statistics)
+            throws Exception;
+
+    /*
+     * Client Job Control methods.
+     */
+    public UUID createJob(JobSpecification jobSpec) throws Exception;
+
+    public UUID createJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
+
+    public JobStatus getJobStatus(UUID jobId) throws Exception;
+
+    public void start(UUID jobId) throws Exception;
+
+    public JobStatistics waitForCompletion(UUID jobId) throws Exception;
+    
+    public Map<String,INodeController> getRegistry() throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/controller/INodeController.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/controller/INodeController.java
new file mode 100644
index 0000000..c271356
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/controller/INodeController.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.controller;
+
+import java.rmi.Remote;
+import java.util.Map;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.comm.Endpoint;
+import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobStage;
+import edu.uci.ics.hyracks.config.NCConfig;
+
+public interface INodeController extends Remote {
+    public String getId() throws Exception;
+    
+    public NCConfig getConfiguration() throws Exception;
+
+    public NodeCapability getNodeCapability() throws Exception;
+
+    public Map<PortInstanceId, Endpoint> initializeJobletPhase1(UUID jobId, JobPlan plan, JobStage stage)
+            throws Exception;
+
+    public void initializeJobletPhase2(UUID jobId, JobPlan plan, JobStage stage,
+            Map<PortInstanceId, Endpoint> globalPortMap) throws Exception;
+
+    public void commitJobletInitialization(UUID jobId, JobPlan plan, JobStage stage) throws Exception;
+
+    public void cleanUpJob(UUID jobId) throws Exception;
+
+    public void startStage(UUID jobId, UUID stageId) throws Exception;
+
+    public void notifyRegistration(IClusterController ccs) throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/controller/NodeCapability.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/controller/NodeCapability.java
new file mode 100644
index 0000000..62d627c
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/controller/NodeCapability.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.controller;
+
+import java.io.Serializable;
+
+public class NodeCapability implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private int cpuCount;
+
+    public int getCPUCount() {
+        return cpuCount;
+    }
+
+    public void setCPUCount(int cpuCount) {
+        this.cpuCount = cpuCount;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityNodeId.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityNodeId.java
new file mode 100644
index 0000000..12f76fe
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityNodeId.java
@@ -0,0 +1,58 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+public final class ActivityNodeId implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private final OperatorDescriptorId odId;
+    private final UUID id;
+
+    public ActivityNodeId(OperatorDescriptorId odId, UUID id) {
+        this.odId = odId;
+        this.id = id;
+    }
+
+    public OperatorDescriptorId getOperatorDescriptorId() {
+        return odId;
+    }
+
+    public UUID getLocalId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        return odId.hashCode() + id.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (!(o instanceof ActivityNodeId)) {
+            return false;
+        }
+        ActivityNodeId other = (ActivityNodeId) o;
+        return other.odId.equals(odId) && other.id.equals(id);
+    }
+
+    public String toString() {
+        return "ANID:" + id;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java
new file mode 100644
index 0000000..c7f3175
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataflow;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+public final class ConnectorDescriptorId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private UUID id;
+
+    public ConnectorDescriptorId(UUID id) {
+        this.id = id;
+    }
+
+    public UUID getId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        return id.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (!(obj instanceof ConnectorDescriptorId)) {
+            return false;
+        }
+        ConnectorDescriptorId other = (ConnectorDescriptorId) obj;
+        return id.equals(other.id);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java
new file mode 100644
index 0000000..c6bc5e3
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataflow;
+
+public interface IActivityGraphBuilder {
+    public void addTask(IActivityNode task);
+
+    public void addBlockingEdge(IActivityNode blocker, IActivityNode blocked);
+
+    public void addSourceEdge(int operatorInputIndex, IActivityNode task, int taskInputIndex);
+
+    public void addTargetEdge(int operatorOutputIndex, IActivityNode task, int taskOutputIndex);
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.java
new file mode 100644
index 0000000..5b57ee3
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.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.dataflow;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public interface IActivityNode extends Serializable {
+    public ActivityNodeId getActivityNodeId();
+
+    public IOperatorDescriptor getOwner();
+
+    public boolean supportsPushInterface();
+
+    public boolean supportsPullInterface();
+
+    public IOperatorNodePushable createPushRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition);
+
+    public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition);
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
new file mode 100644
index 0000000..600fd41
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+/**
+ * Connector that connects operators in a Job.
+ * 
+ * @author vinayakb
+ */
+public interface IConnectorDescriptor extends Serializable {
+    /**
+     * Gets the id of the connector.
+     * 
+     * @return
+     */
+    public ConnectorDescriptorId getConnectorId();
+
+    /**
+     * Factory method to create the send side writer that writes into this connector.
+     * 
+     * @param ctx
+     *            Context
+     * @param plan
+     *            Job plan
+     * @param edwFactory
+     *            Endpoint writer factory.
+     * @param index
+     *            ordinal index of the data producer partition.
+     * @return data writer.
+     * @throws Exception
+     */
+    public IFrameWriter createSendSideWriter(HyracksContext ctx, JobPlan plan, IEndpointDataWriterFactory edwFactory,
+            int index) throws HyracksDataException;
+
+    /**
+     * Factory metod to create the receive side reader that reads data from this connector.
+     * 
+     * @param ctx
+     *            Context
+     * @param plan
+     *            Job plan
+     * @param demux
+     *            Connection Demultiplexer
+     * @param index
+     *            ordinal index of the data consumer partition
+     * @return data reader
+     * @throws HyracksDataException
+     */
+    public IFrameReader createReceiveSideReader(HyracksContext ctx, JobPlan plan, IConnectionDemultiplexer demux,
+            int index) throws HyracksDataException;
+
+    /**
+     * Translate this connector descriptor to JSON.
+     * 
+     * @return
+     * @throws JSONException
+     */
+    public JSONObject toJSON() throws JSONException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IDataReader.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IDataReader.java
new file mode 100644
index 0000000..b5f45cb
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IDataReader.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataflow;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Provides data to a consumer.
+ * 
+ * @author vinayakb
+ */
+public interface IDataReader<T> {
+    /**
+     * Reads one data item.
+     * 
+     * @return data. <code>null</code> indicates end of stream.
+     * @throws Exception
+     */
+    public T readData() throws HyracksDataException;
+
+    /**
+     * Closes this reader.
+     * 
+     * @throws Exception
+     */
+    public void close() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IDataWriter.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IDataWriter.java
new file mode 100644
index 0000000..128576c
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IDataWriter.java
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Accepts data from data producers.
+ * 
+ * @author vinayakb
+ */
+public interface IDataWriter<T> {
+    /**
+     * Pushes data to the acceptor.
+     * 
+     * @param data
+     *            - Data pushed to the acceptor. <code>null</code> indicates the end of stream.
+     * @throws HyracksDataException
+     */
+    public void writeData(T data) throws HyracksDataException;
+
+    /**
+     * Closes this writer.
+     * 
+     * @throws Exception
+     */
+    public void close() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
new file mode 100644
index 0000000..45c589a
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.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.dataflow;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IEndpointDataWriterFactory {
+    public IFrameWriter createFrameWriter(int index) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOpenableDataReader.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOpenableDataReader.java
new file mode 100644
index 0000000..d0b6474
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOpenableDataReader.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.dataflow;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IOpenableDataReader<T> extends IDataReader<T> {
+    public void open() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOpenableDataWriter.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOpenableDataWriter.java
new file mode 100644
index 0000000..b2b15c6
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOpenableDataWriter.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.dataflow;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IOpenableDataWriter<T> extends IDataWriter<T> {
+    public void open() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
new file mode 100644
index 0000000..17a0fc5
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
@@ -0,0 +1,101 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+/**
+ * Descriptor for operators in Hyracks.
+ * 
+ * @author vinayakb
+ */
+public interface IOperatorDescriptor extends Serializable {
+    /**
+     * Returns the id of the operator.
+     * 
+     * @return operator id
+     */
+    public OperatorDescriptorId getOperatorId();
+
+    /**
+     * Returns the number of inputs into this operator.
+     * 
+     * @return Number of inputs.
+     */
+    public int getInputArity();
+
+    /**
+     * Returns the number of outputs out of this operator.
+     * 
+     * @return Number of outputs.
+     */
+    public int getOutputArity();
+
+    /**
+     * Returns the partition constraint requested for scheduling this operator. This value is set as part of Job creation by the client.
+     * 
+     * @return the partition constraint.
+     */
+    public PartitionConstraint getPartitionConstraint();
+
+    /**
+     * Sets the partition constraint.
+     * 
+     * @param partitionConstraint
+     *            - partition constraint.
+     */
+    public void setPartitionConstraint(PartitionConstraint partitionConstraint);
+
+    /**
+     * Returns the final partition locations selected for scheduling. These are decided by Hyracks such that they satisfy the partition constraints.
+     * 
+     * @return array indicating number and node ids of the nodes to schedule the operator runtimes.
+     */
+    public String[] getPartitions();
+
+    /**
+     * Sets the partition locations.
+     * 
+     * @param partitions
+     *            node ids to schedule the operators.
+     */
+    public void setPartitions(String[] partitions);
+
+    /**
+     * Gets the output record descriptor
+     * 
+     * @return Array of RecordDescriptor, one per output.
+     */
+    public RecordDescriptor[] getOutputRecordDescriptors();
+
+    /**
+     * Contributes the activity graph that describes the behavior of this operator.
+     * 
+     * @param builder
+     *            - graph builder
+     */
+    public void contributeTaskGraph(IActivityGraphBuilder builder);
+
+    /**
+     * Translates this operator descriptor to JSON.
+     */
+    public JSONObject toJSON() throws JSONException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePullable.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePullable.java
new file mode 100644
index 0000000..1fa039f
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePullable.java
@@ -0,0 +1,19 @@
+/*
+ * 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;
+
+public interface IOperatorNodePullable extends IOpenableDataReader<Object[]> {
+    public void setDataReader(int index, IOpenableDataWriter<Object[]> writer);
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePushable.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePushable.java
new file mode 100644
index 0000000..13ac15c
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorNodePushable.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.dataflow;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+
+public interface IOperatorNodePushable extends IFrameWriter {
+    public void setFrameWriter(int index, IFrameWriter writer);
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
new file mode 100644
index 0000000..bbf3cf5
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.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.api.dataflow;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+public final class OperatorDescriptorId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final UUID id;
+
+    public OperatorDescriptorId(UUID id) {
+        this.id = id;
+    }
+
+    public UUID getId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        return id.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof OperatorDescriptorId)) {
+            return false;
+        }
+        return ((OperatorDescriptorId) o).id.equals(id);
+    }
+
+    @Override
+    public String toString() {
+        return "ODID:" + id;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorInstanceId.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorInstanceId.java
new file mode 100644
index 0000000..f6ee39b
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorInstanceId.java
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+
+public final class OperatorInstanceId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private OperatorDescriptorId odId;
+    private int partition;
+
+    public OperatorInstanceId(OperatorDescriptorId odId, int partition) {
+        this.odId = odId;
+        this.partition = partition;
+    }
+
+    public OperatorDescriptorId getOperatorId() {
+        return odId;
+    }
+
+    public int getPartition() {
+        return partition;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((odId == null) ? 0 : odId.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;
+        OperatorInstanceId other = (OperatorInstanceId) obj;
+        if (odId == null) {
+            if (other.odId != null)
+                return false;
+        } else if (!odId.equals(other.odId))
+            return false;
+        if (partition != other.partition)
+            return false;
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/PortInstanceId.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/PortInstanceId.java
new file mode 100644
index 0000000..d8057d9
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/PortInstanceId.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataflow;
+
+import java.io.Serializable;
+
+public final class PortInstanceId implements Serializable {
+    public enum Direction {
+        INPUT,
+        OUTPUT,
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    private OperatorDescriptorId odId;
+    private Direction direction;
+    private int portIndex;
+    private int partition;
+
+    public PortInstanceId(OperatorDescriptorId odId, Direction direction, int portIndex, int partition) {
+        this.odId = odId;
+        this.direction = direction;
+        this.portIndex = portIndex;
+        this.partition = partition;
+    }
+
+    public OperatorDescriptorId getOperatorId() {
+        return odId;
+    }
+
+    public Direction getDirection() {
+        return direction;
+    }
+
+    public int getPortIndex() {
+        return portIndex;
+    }
+
+    public int getPartition() {
+        return partition;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((direction == null) ? 0 : direction.hashCode());
+        result = prime * result + ((odId == null) ? 0 : odId.hashCode());
+        result = prime * result + partition;
+        result = prime * result + portIndex;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        PortInstanceId other = (PortInstanceId) obj;
+        if (direction == null) {
+            if (other.direction != null)
+                return false;
+        } else if (!direction.equals(other.direction))
+            return false;
+        if (odId == null) {
+            if (other.odId != null)
+                return false;
+        } else if (!odId.equals(other.odId))
+            return false;
+        if (partition != other.partition)
+            return false;
+        if (portIndex != other.portIndex)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return odId + ":" + direction + ":" + partition + ":" + portIndex;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IBinaryComparator.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IBinaryComparator.java
new file mode 100644
index 0000000..767684d
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IBinaryComparator.java
@@ -0,0 +1,19 @@
+/*
+ * 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;
+
+public interface IBinaryComparator {
+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2);
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IBinaryComparatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IBinaryComparatorFactory.java
new file mode 100644
index 0000000..9976818
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IBinaryComparatorFactory.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.dataflow.value;
+
+import java.io.Serializable;
+
+public interface IBinaryComparatorFactory extends Serializable {
+    public IBinaryComparator createBinaryComparator();
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IBinaryHashFunction.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IBinaryHashFunction.java
new file mode 100644
index 0000000..a995703
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IBinaryHashFunction.java
@@ -0,0 +1,19 @@
+/*
+ * 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;
+
+public interface IBinaryHashFunction {
+    int hash(byte[] bytes, int offset, int length);
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IBinaryHashFunctionFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..a064d5f
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IBinaryHashFunctionFactory.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.dataflow.value;
+
+import java.io.Serializable;
+
+public interface IBinaryHashFunctionFactory extends Serializable {
+    public IBinaryHashFunction createBinaryHashFunction();
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IComparator.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IComparator.java
new file mode 100644
index 0000000..2d24496
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IComparator.java
@@ -0,0 +1,19 @@
+/*
+ * 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;
+
+public interface IComparator<T> {
+    public int compare(T o1, T o2);
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IComparatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IComparatorFactory.java
new file mode 100644
index 0000000..ef9161f
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IComparatorFactory.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.dataflow.value;
+
+import java.io.Serializable;
+
+public interface IComparatorFactory<T> extends Serializable {
+    public IComparator<T> createComparator();
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IHashFunction.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IHashFunction.java
new file mode 100644
index 0000000..9124fca
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IHashFunction.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.dataflow.value;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IHashFunction<T> {
+    public int hash(T o) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IHashFunctionFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IHashFunctionFactory.java
new file mode 100644
index 0000000..b67a96c
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IHashFunctionFactory.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.dataflow.value;
+
+import java.io.Serializable;
+
+public interface IHashFunctionFactory<T> extends Serializable {
+    public IHashFunction<T> createHashFunction();
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ISerializerDeserializer.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ISerializerDeserializer.java
new file mode 100644
index 0000000..33ba602
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ISerializerDeserializer.java
@@ -0,0 +1,42 @@
+/*
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ISerializerDeserializer<T> extends Serializable {
+    /**
+     * Deserialization method.
+     * 
+     * @param in
+     *            - Stream to read instance from.
+     * @return A new instance of T with data.
+     */
+    public T deserialize(DataInput in) throws HyracksDataException;
+
+    /**
+     * Serialization method.
+     * 
+     * @param instance
+     *            - the instance to serialize.
+     * @param out
+     *            - Stream to write data to.
+     */
+    public void serialize(T instance, DataOutput out) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ITuplePartitionComputer.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ITuplePartitionComputer.java
new file mode 100644
index 0000000..d9b023d
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ITuplePartitionComputer.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.dataflow.value;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+
+public interface ITuplePartitionComputer {
+    public int partition(FrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java
new file mode 100644
index 0000000..eb2bbe8
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.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.dataflow.value;
+
+import java.io.Serializable;
+
+public interface ITuplePartitionComputerFactory extends Serializable {
+    public ITuplePartitionComputer createPartitioner();
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/RecordDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/RecordDescriptor.java
new file mode 100644
index 0000000..20dc034
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/RecordDescriptor.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataflow.value;
+
+import java.io.Serializable;
+
+@SuppressWarnings("unchecked")
+public final class RecordDescriptor implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final ISerializerDeserializer[] fields;
+
+    public RecordDescriptor(ISerializerDeserializer[] fields) {
+        this.fields = fields;
+    }
+
+    public ISerializerDeserializer[] getFields() {
+        return fields;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/exceptions/HyracksDataException.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/exceptions/HyracksDataException.java
new file mode 100644
index 0000000..1bdfe4b
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/exceptions/HyracksDataException.java
@@ -0,0 +1,36 @@
+/*
+ * 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.exceptions;
+
+import java.io.IOException;
+
+public class HyracksDataException extends IOException {
+    private static final long serialVersionUID = 1L;
+
+    public HyracksDataException() {
+    }
+
+    public HyracksDataException(String message) {
+        super(message);
+    }
+
+    public HyracksDataException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public HyracksDataException(Throwable cause) {
+        super(cause);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/IOperatorEnvironment.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/IOperatorEnvironment.java
new file mode 100644
index 0000000..b41cb35
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/IOperatorEnvironment.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.job;
+
+public interface IOperatorEnvironment {
+    public void set(String name, Object value);
+
+    public Object get(String name);
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobFlag.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobFlag.java
new file mode 100644
index 0000000..9219ec3
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobFlag.java
@@ -0,0 +1,19 @@
+/*
+ * 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.job;
+
+public enum JobFlag {
+    COLLECT_FRAME_COUNTS
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobPlan.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobPlan.java
new file mode 100644
index 0000000..8c71438
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobPlan.java
@@ -0,0 +1,164 @@
+/*
+ * 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.job;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class JobPlan implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final JobSpecification jobSpec;
+
+    private final EnumSet<JobFlag> jobFlags;
+
+    private final Map<ActivityNodeId, IActivityNode> activityNodes;
+
+    private final Map<ActivityNodeId, Set<ActivityNodeId>> blocker2blockedMap;
+
+    private final Map<ActivityNodeId, Set<ActivityNodeId>> blocked2blockerMap;
+
+    private final Map<OperatorDescriptorId, Set<ActivityNodeId>> operatorTaskMap;
+
+    private final Map<ActivityNodeId, List<Integer>> taskInputMap;
+
+    private final Map<ActivityNodeId, List<Integer>> taskOutputMap;
+
+    private final Map<OperatorDescriptorId, List<ActivityNodeId>> operatorInputMap;
+
+    private final Map<OperatorDescriptorId, List<ActivityNodeId>> operatorOutputMap;
+
+    private JobStage endStage;
+
+    public JobPlan(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
+        this.jobSpec = jobSpec;
+        this.jobFlags = jobFlags;
+        activityNodes = new HashMap<ActivityNodeId, IActivityNode>();
+        blocker2blockedMap = new HashMap<ActivityNodeId, Set<ActivityNodeId>>();
+        blocked2blockerMap = new HashMap<ActivityNodeId, Set<ActivityNodeId>>();
+        operatorTaskMap = new HashMap<OperatorDescriptorId, Set<ActivityNodeId>>();
+        taskInputMap = new HashMap<ActivityNodeId, List<Integer>>();
+        taskOutputMap = new HashMap<ActivityNodeId, List<Integer>>();
+        operatorInputMap = new HashMap<OperatorDescriptorId, List<ActivityNodeId>>();
+        operatorOutputMap = new HashMap<OperatorDescriptorId, List<ActivityNodeId>>();
+    }
+
+    public JobSpecification getJobSpecification() {
+        return jobSpec;
+    }
+
+    public EnumSet<JobFlag> getJobFlags() {
+        return jobFlags;
+    }
+
+    public Map<ActivityNodeId, IActivityNode> getActivityNodeMap() {
+        return activityNodes;
+    }
+
+    public Map<ActivityNodeId, Set<ActivityNodeId>> getBlocker2BlockedMap() {
+        return blocker2blockedMap;
+    }
+
+    public Map<ActivityNodeId, Set<ActivityNodeId>> getBlocked2BlockerMap() {
+        return blocked2blockerMap;
+    }
+
+    public Map<OperatorDescriptorId, Set<ActivityNodeId>> getOperatorTaskMap() {
+        return operatorTaskMap;
+    }
+
+    public Map<ActivityNodeId, List<Integer>> getTaskInputMap() {
+        return taskInputMap;
+    }
+
+    public Map<ActivityNodeId, List<Integer>> getTaskOutputMap() {
+        return taskOutputMap;
+    }
+
+    public Map<OperatorDescriptorId, List<ActivityNodeId>> getOperatorInputMap() {
+        return operatorInputMap;
+    }
+
+    public Map<OperatorDescriptorId, List<ActivityNodeId>> getOperatorOutputMap() {
+        return operatorOutputMap;
+    }
+
+    public void setEndStage(JobStage endStage) {
+        this.endStage = endStage;
+    }
+
+    public JobStage getEndStage() {
+        return endStage;
+    }
+
+    public List<IConnectorDescriptor> getTaskInputs(ActivityNodeId hanId) {
+        List<Integer> inputIndexes = taskInputMap.get(hanId);
+        if (inputIndexes == null) {
+            return null;
+        }
+        OperatorDescriptorId ownerId = hanId.getOperatorDescriptorId();
+        List<IConnectorDescriptor> inputs = new ArrayList<IConnectorDescriptor>();
+        for (Integer i : inputIndexes) {
+            inputs.add(jobSpec.getInputConnectorDescriptor(ownerId, i));
+        }
+        return inputs;
+    }
+
+    public List<IConnectorDescriptor> getTaskOutputs(ActivityNodeId hanId) {
+        List<Integer> outputIndexes = taskOutputMap.get(hanId);
+        if (outputIndexes == null) {
+            return null;
+        }
+        OperatorDescriptorId ownerId = hanId.getOperatorDescriptorId();
+        List<IConnectorDescriptor> outputs = new ArrayList<IConnectorDescriptor>();
+        for (Integer i : outputIndexes) {
+            outputs.add(jobSpec.getOutputConnectorDescriptor(ownerId, i));
+        }
+        return outputs;
+    }
+
+    public RecordDescriptor getTaskInputRecordDescriptor(ActivityNodeId hanId, int inputIndex) {
+        int opInputIndex = getTaskInputMap().get(hanId).get(inputIndex);
+        return jobSpec.getOperatorInputRecordDescriptor(hanId.getOperatorDescriptorId(), opInputIndex);
+    }
+
+    public RecordDescriptor getTaskOutputRecordDescriptor(ActivityNodeId hanId, int outputIndex) {
+        int opOutputIndex = getTaskOutputMap().get(hanId).get(outputIndex);
+        return jobSpec.getOperatorOutputRecordDescriptor(hanId.getOperatorDescriptorId(), opOutputIndex);
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder buffer = new StringBuilder();
+        buffer.append("ActivityNodes: " + activityNodes);
+        buffer.append('\n');
+        buffer.append("Blocker->Blocked: " + blocker2blockedMap);
+        buffer.append('\n');
+        buffer.append("Blocked->Blocker: " + blocked2blockerMap);
+        buffer.append('\n');
+        return buffer.toString();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
new file mode 100644
index 0000000..27bbdc8
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
@@ -0,0 +1,228 @@
+/*
+ * 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.job;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.util.Pair;
+
+public class JobSpecification implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final List<OperatorDescriptorId> roots;
+
+    private final Map<OperatorDescriptorId, IOperatorDescriptor> opMap;
+
+    private final Map<ConnectorDescriptorId, IConnectorDescriptor> connMap;
+
+    private final Map<OperatorDescriptorId, List<IConnectorDescriptor>> opInputMap;
+
+    private final Map<OperatorDescriptorId, List<IConnectorDescriptor>> opOutputMap;
+
+    private final Map<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> connectorOpMap;
+
+    public JobSpecification() {
+        roots = new ArrayList<OperatorDescriptorId>();
+        opMap = new HashMap<OperatorDescriptorId, IOperatorDescriptor>();
+        connMap = new HashMap<ConnectorDescriptorId, IConnectorDescriptor>();
+        opInputMap = new HashMap<OperatorDescriptorId, List<IConnectorDescriptor>>();
+        opOutputMap = new HashMap<OperatorDescriptorId, List<IConnectorDescriptor>>();
+        connectorOpMap = new HashMap<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>>();
+    }
+
+    public void addRoot(IOperatorDescriptor op) {
+        roots.add(op.getOperatorId());
+    }
+
+    public void connect(IConnectorDescriptor conn, IOperatorDescriptor producerOp, int producerPort,
+            IOperatorDescriptor consumerOp, int consumerPort) {
+        insertIntoIndexedMap(opInputMap, consumerOp.getOperatorId(), consumerPort, conn);
+        insertIntoIndexedMap(opOutputMap, producerOp.getOperatorId(), producerPort, conn);
+        connectorOpMap.put(conn.getConnectorId(),
+                new Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>(
+                        new Pair<IOperatorDescriptor, Integer>(producerOp, producerPort),
+                        new Pair<IOperatorDescriptor, Integer>(consumerOp, consumerPort)));
+    }
+
+    private <T> void extend(List<T> list, int index) {
+        int n = list.size();
+        for (int i = n; i <= index; ++i) {
+            list.add(null);
+        }
+    }
+
+    public Map<ConnectorDescriptorId, IConnectorDescriptor> getConnectorMap() {
+        return connMap;
+    }
+
+    public Map<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> getConnectorOperatorMap() {
+        return connectorOpMap;
+    }
+
+    public RecordDescriptor getConnectorRecordDescriptor(IConnectorDescriptor conn) {
+        Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
+                .getConnectorId());
+        return connInfo.first.first.getOutputRecordDescriptors()[connInfo.first.second];
+    }
+
+    public IOperatorDescriptor getConsumer(IConnectorDescriptor conn) {
+        Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
+                .getConnectorId());
+        return connInfo.second.first;
+    }
+
+    public int getConsumerInputIndex(IConnectorDescriptor conn) {
+        Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
+                .getConnectorId());
+        return connInfo.second.second;
+    }
+
+    public IConnectorDescriptor getInputConnectorDescriptor(IOperatorDescriptor op, int inputIndex) {
+        return getInputConnectorDescriptor(op.getOperatorId(), inputIndex);
+    }
+
+    public IConnectorDescriptor getInputConnectorDescriptor(OperatorDescriptorId odId, int inputIndex) {
+        return opInputMap.get(odId).get(inputIndex);
+    }
+
+    public Map<OperatorDescriptorId, List<IConnectorDescriptor>> getOperatorInputMap() {
+        return opInputMap;
+    }
+
+    public RecordDescriptor getOperatorInputRecordDescriptor(OperatorDescriptorId odId, int inputIndex) {
+        return getConnectorRecordDescriptor(getInputConnectorDescriptor(odId, inputIndex));
+    }
+
+    public Map<OperatorDescriptorId, IOperatorDescriptor> getOperatorMap() {
+        return opMap;
+    }
+
+    public Map<OperatorDescriptorId, List<IConnectorDescriptor>> getOperatorOutputMap() {
+        return opOutputMap;
+    }
+
+    public RecordDescriptor getOperatorOutputRecordDescriptor(OperatorDescriptorId odId, int outputIndex) {
+        return getConnectorRecordDescriptor(getOutputConnectorDescriptor(odId, outputIndex));
+    }
+
+    public IConnectorDescriptor getOutputConnectorDescriptor(IOperatorDescriptor op, int outputIndex) {
+        return getOutputConnectorDescriptor(op.getOperatorId(), outputIndex);
+    }
+
+    public IConnectorDescriptor getOutputConnectorDescriptor(OperatorDescriptorId odId, int outputIndex) {
+        return opOutputMap.get(odId).get(outputIndex);
+    }
+
+    public IOperatorDescriptor getProducer(IConnectorDescriptor conn) {
+        Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
+                .getConnectorId());
+        return connInfo.first.first;
+    }
+
+    public int getProducerOutputIndex(IConnectorDescriptor conn) {
+        Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connInfo = connectorOpMap.get(conn
+                .getConnectorId());
+        return connInfo.first.second;
+    }
+
+    public List<OperatorDescriptorId> getRoots() {
+        return roots;
+    }
+
+    private <K, V> void insertIntoIndexedMap(Map<K, List<V>> map, K key, int index, V value) {
+        List<V> vList = map.get(key);
+        if (vList == null) {
+            vList = new ArrayList<V>();
+            map.put(key, vList);
+        }
+        extend(vList, index);
+        vList.set(index, value);
+    }
+
+    public String toString() {
+        StringBuilder buffer = new StringBuilder();
+
+        for (Map.Entry<OperatorDescriptorId, IOperatorDescriptor> e : opMap.entrySet()) {
+            buffer.append(e.getKey().getId()).append(" : ").append(e.getValue().toString()).append("\n");
+            IOperatorDescriptor op = e.getValue();
+            buffer.append("   Partition Constraint: ").append(op.getPartitionConstraint()).append("\n");
+            List<IConnectorDescriptor> inputs = opInputMap.get(e.getKey());
+            if (inputs != null && !inputs.isEmpty()) {
+                buffer.append("   Inputs:\n");
+                for (IConnectorDescriptor c : inputs) {
+                    buffer.append("      ").append(c.getConnectorId().getId()).append(" : ").append(c.toString())
+                            .append("\n");
+                }
+            }
+            List<IConnectorDescriptor> outputs = opOutputMap.get(e.getKey());
+            if (outputs != null && !outputs.isEmpty()) {
+                buffer.append("   Outputs:\n");
+                for (IConnectorDescriptor c : outputs) {
+                    buffer.append("      ").append(c.getConnectorId().getId()).append(" : ").append(c.toString())
+                            .append("\n");
+                }
+            }
+        }
+
+        return buffer.toString();
+    }
+
+    public JSONObject toJSON() throws JSONException {
+        JSONObject jjob = new JSONObject();
+
+        jjob.put("type", "job");
+
+        JSONArray jopArray = new JSONArray();
+        for (Map.Entry<OperatorDescriptorId, IOperatorDescriptor> e : opMap.entrySet()) {
+            jopArray.put(e.getValue().toJSON());
+        }
+        jjob.put("operators", jopArray);
+
+        JSONArray jcArray = new JSONArray();
+        for (Map.Entry<ConnectorDescriptorId, IConnectorDescriptor> e : connMap.entrySet()) {
+            jcArray.put(e.getValue().toJSON());
+        }
+        jjob.put("connectors", jcArray);
+
+        JSONArray jconnectionsArray = new JSONArray();
+        for (Map.Entry<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> e : connectorOpMap
+                .entrySet()) {
+            JSONObject conn = new JSONObject();
+            conn.put("type", "connection");
+            conn.put("connector-id", e.getValue().toString());
+            conn.put("in-operator-id", e.getValue().first.first.getOperatorId().toString());
+            conn.put("in-operator-port", e.getValue().first.second.intValue());
+            conn.put("out-operator-id", e.getValue().second.first.getOperatorId().toString());
+            conn.put("out-operator-port", e.getValue().second.second.intValue());
+            jconnectionsArray.put(conn);
+        }
+        jjob.put("connections", jconnectionsArray);
+
+        return jjob;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobStage.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobStage.java
new file mode 100644
index 0000000..41187f1
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobStage.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.job;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+
+public class JobStage implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final UUID id;
+
+    private final Set<ActivityNodeId> tasks;
+
+    private final Set<JobStage> dependencies;
+
+    private final Set<JobStage> dependents;
+
+    private boolean started;
+
+    public JobStage(Set<ActivityNodeId> tasks) {
+        this.id = UUID.randomUUID();
+        this.tasks = tasks;
+        dependencies = new HashSet<JobStage>();
+        dependents = new HashSet<JobStage>();
+    }
+
+    public UUID getId() {
+        return id;
+    }
+
+    public Set<ActivityNodeId> getTasks() {
+        return tasks;
+    }
+
+    public void addDependency(JobStage stage) {
+        dependencies.add(stage);
+    }
+
+    public void addDependent(JobStage stage) {
+        dependents.add(stage);
+    }
+
+    public Set<JobStage> getDependencies() {
+        return dependencies;
+    }
+
+    @Override
+    public int hashCode() {
+        return id == null ? 0 : id.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof JobStage)) {
+            return false;
+        }
+        return id == ((JobStage) o).id;
+    }
+
+    @Override
+    public String toString() {
+        return "SID:" + id + ": " + tasks;
+    }
+
+    public boolean isStarted() {
+        return started;
+    }
+
+    public void setStarted() {
+        started = true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobStatus.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobStatus.java
new file mode 100644
index 0000000..977f41a
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/JobStatus.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.job;
+
+public enum JobStatus {
+    INITIALIZED,
+    RUNNING,
+    TERMINATED,
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/statistics/JobStatistics.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/statistics/JobStatistics.java
new file mode 100644
index 0000000..921b239
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/statistics/JobStatistics.java
@@ -0,0 +1,91 @@
+/*
+ * 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.job.statistics;
+
+import java.io.Serializable;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+public class JobStatistics implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private Date startTime;
+
+    private Date endTime;
+
+    private List<StageStatistics> stages;
+
+    public JobStatistics() {
+        stages = new ArrayList<StageStatistics>();
+    }
+
+    public void setStartTime(Date startTime) {
+        this.startTime = startTime;
+    }
+
+    public Date getStartTime() {
+        return startTime;
+    }
+
+    public void setEndTime(Date endTime) {
+        this.endTime = endTime;
+    }
+
+    public Date getEndTime() {
+        return endTime;
+    }
+
+    public void addStageStatistics(StageStatistics stageStatistics) {
+        stages.add(stageStatistics);
+    }
+
+    public List<StageStatistics> getStages() {
+        return stages;
+    }
+
+    @Override
+    public String toString() {
+        DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
+        StringBuilder buffer = new StringBuilder();
+
+        buffer.append("{\n");
+        indent(buffer, 1).append("startTime: '").append(df.format(startTime)).append("',\n");
+        indent(buffer, 1).append("endTime: '").append(df.format(endTime)).append("',\n");
+        indent(buffer, 1).append("stages: [\n");
+        boolean first = true;
+        for (StageStatistics ss : stages) {
+            if (!first) {
+                buffer.append(",\n");
+            }
+            first = false;
+            ss.toString(buffer, 2);
+        }
+        buffer.append("\n");
+        indent(buffer, 1).append("]\n");
+        buffer.append("}");
+
+        return buffer.toString();
+    }
+
+    static StringBuilder indent(StringBuilder buffer, int level) {
+        for (int i = 0; i < level; ++i) {
+            buffer.append(" ");
+        }
+        return buffer;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/statistics/StageStatistics.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/statistics/StageStatistics.java
new file mode 100644
index 0000000..6cc437b
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/statistics/StageStatistics.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.job.statistics;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+public class StageStatistics implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private UUID stageId;
+
+    private Map<String, StageletStatistics> stageletMap;
+
+    public StageStatistics() {
+        stageletMap = new HashMap<String, StageletStatistics>();
+    }
+
+    public void setStageId(UUID stageId) {
+        this.stageId = stageId;
+    }
+
+    public UUID getStageId() {
+        return stageId;
+    }
+
+    public void addStageletStatistics(StageletStatistics ss) {
+        stageletMap.put(ss.getNodeId(), ss);
+    }
+
+    public Map<String, StageletStatistics> getStageletStatistics() {
+        return stageletMap;
+    }
+
+    public void toString(StringBuilder buffer, int level) {
+        JobStatistics.indent(buffer, level).append("{\n");
+        JobStatistics.indent(buffer, level + 1).append("stageId: '").append(stageId).append("',\n");
+        JobStatistics.indent(buffer, level + 1).append("stagelets: {\n");
+        boolean first = true;
+        for (Map.Entry<String, StageletStatistics> e : stageletMap.entrySet()) {
+            if (!first) {
+                buffer.append(",\n");
+            }
+            first = false;
+            JobStatistics.indent(buffer, level + 2).append(e.getKey()).append(": ");
+            e.getValue().toString(buffer, level + 3);
+        }
+        buffer.append("\n");
+        JobStatistics.indent(buffer, level + 1).append("}\n");
+        JobStatistics.indent(buffer, level).append("}");
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/statistics/StageletStatistics.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/statistics/StageletStatistics.java
new file mode 100644
index 0000000..66fb871
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/job/statistics/StageletStatistics.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.job.statistics;
+
+import java.io.Serializable;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Map;
+import java.util.TreeMap;
+
+public class StageletStatistics implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private String nodeId;
+
+    private Date startTime;
+
+    private Date endTime;
+
+    private Map<String, String> statisticsMap;
+
+    public StageletStatistics() {
+        statisticsMap = Collections.synchronizedMap(new TreeMap<String, String>());
+    }
+
+    public void setNodeId(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public void setStartTime(Date startTime) {
+        this.startTime = startTime;
+    }
+
+    public Date getStartTime() {
+        return startTime;
+    }
+
+    public void setEndTime(Date endTime) {
+        this.endTime = endTime;
+    }
+
+    public Date getEndTime() {
+        return endTime;
+    }
+
+    public Map<String, String> getStatisticsMap() {
+        return statisticsMap;
+    }
+
+    public void toString(StringBuilder buffer, int level) {
+        DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
+        buffer.append("{\n");
+        JobStatistics.indent(buffer, level + 1).append("nodeId: '").append(nodeId).append("',\n");
+        JobStatistics.indent(buffer, level + 1).append("startTime: '").append(df.format(startTime)).append("',\n");
+        JobStatistics.indent(buffer, level + 1).append("endTime: '").append(df.format(endTime)).append("',\n");
+        JobStatistics.indent(buffer, level + 1).append("statistics: {\n");
+        boolean first = true;
+        for (Map.Entry<String, String> e : statisticsMap.entrySet()) {
+            if (!first) {
+                buffer.append(",\n");
+            }
+            first = false;
+            JobStatistics.indent(buffer, level + 2).append(e.getKey()).append(": '").append(e.getValue()).append("'");
+        }
+        buffer.append("\n");
+        JobStatistics.indent(buffer, level + 1).append("}\n");
+        JobStatistics.indent(buffer, level).append("}");
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/util/Pair.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/util/Pair.java
new file mode 100644
index 0000000..2364d74
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/api/util/Pair.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.util;
+
+import java.io.Serializable;
+
+public final class Pair<F, S> implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public F first;
+    public S second;
+
+    public Pair(F first, S second) {
+        this.first = first;
+        this.second = second;
+    }
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/ConnectionEntry.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/ConnectionEntry.java
new file mode 100644
index 0000000..4d9c309
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/ConnectionEntry.java
@@ -0,0 +1,146 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.SocketChannel;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
+import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
+import edu.uci.ics.hyracks.api.comm.IDataReceiveListener;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class ConnectionEntry implements IConnectionEntry {
+    private static final Logger LOGGER = Logger.getLogger(ConnectionEntry.class.getName());
+
+    private SocketChannel socketChannel;
+
+    private final ByteBuffer readBuffer;
+
+    private final ByteBuffer writeBuffer;
+
+    private IDataReceiveListener recvListener;
+
+    private Object attachment;
+
+    private final SelectionKey key;
+
+    public ConnectionEntry(HyracksContext ctx, SocketChannel socketChannel, SelectionKey key) {
+        this.socketChannel = socketChannel;
+        readBuffer = ctx.getResourceManager().allocateFrame();
+        readBuffer.clear();
+        writeBuffer = ctx.getResourceManager().allocateFrame();
+        writeBuffer.clear();
+        this.key = key;
+    }
+
+    public SocketChannel getSocketChannel() {
+        return socketChannel;
+    }
+
+    public boolean dispatch(SelectionKey key) throws IOException {
+        if (key.isReadable()) {
+            if (LOGGER.isLoggable(Level.FINER)) {
+                LOGGER.finer("Before read: " + readBuffer.position() + " " + readBuffer.limit());
+            }
+            int bytesRead = socketChannel.read(readBuffer);
+            if (bytesRead < 0) {
+                recvListener.eos(this);
+                return true;
+            }
+            if (LOGGER.isLoggable(Level.FINER)) {
+                LOGGER.finer("After read: " + readBuffer.position() + " " + readBuffer.limit());
+            }
+            recvListener.dataReceived(this);
+        } else if (key.isWritable()) {
+            synchronized (this) {
+                writeBuffer.flip();
+                if (LOGGER.isLoggable(Level.FINER)) {
+                    LOGGER.finer("Before write: " + writeBuffer.position() + " " + writeBuffer.limit());
+                }
+                int bytesWritten = socketChannel.write(writeBuffer);
+                if (bytesWritten < 0) {
+                    return true;
+                }
+                if (LOGGER.isLoggable(Level.FINER)) {
+                    LOGGER.finer("After write: " + writeBuffer.position() + " " + writeBuffer.limit());
+                }
+                if (writeBuffer.remaining() <= 0) {
+                    int ops = key.interestOps();
+                    key.interestOps(ops & ~SelectionKey.OP_WRITE);
+                }
+                writeBuffer.compact();
+                notifyAll();
+            }
+        } else {
+            LOGGER.warning("Spurious event triggered: " + key.readyOps());
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public ByteBuffer getReadBuffer() {
+        return readBuffer;
+    }
+
+    @Override
+    public synchronized void write(ByteBuffer buffer) {
+        while (buffer.remaining() > 0) {
+            while (writeBuffer.remaining() <= 0) {
+                try {
+                    wait();
+                } catch (InterruptedException e) {
+                }
+            }
+            int oldLimit = buffer.limit();
+            buffer.limit(Math.min(oldLimit, writeBuffer.remaining()));
+            writeBuffer.put(buffer);
+            buffer.limit(oldLimit);
+            int ops = key.interestOps();
+            key.interestOps(ops | SelectionKey.OP_WRITE);
+            key.selector().wakeup();
+        }
+    }
+
+    @Override
+    public void setDataReceiveListener(IDataReceiveListener listener) {
+        this.recvListener = listener;
+    }
+
+    @Override
+    public void attach(Object attachment) {
+        this.attachment = attachment;
+    }
+
+    @Override
+    public Object getAttachment() {
+        return attachment;
+    }
+
+    @Override
+    public void close() throws IOException {
+        socketChannel.close();
+    }
+
+    @Override
+    public SelectionKey getSelectionKey() {
+        return key;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/ConnectionManager.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/ConnectionManager.java
new file mode 100644
index 0000000..bfca6ad
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/ConnectionManager.java
@@ -0,0 +1,351 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.nio.ByteBuffer;
+import java.nio.channels.AsynchronousCloseException;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
+import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
+import edu.uci.ics.hyracks.api.comm.IDataReceiveListener;
+import edu.uci.ics.hyracks.api.comm.IDataReceiveListenerFactory;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameHelper;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class ConnectionManager {
+    private static final Logger LOGGER = Logger.getLogger(ConnectionManager.class.getName());
+
+    private static final int INITIAL_MESSAGE_LEN = 20;
+
+    private NetworkAddress networkAddress;
+
+    private ServerSocketChannel serverSocketChannel;
+
+    private final HyracksContext ctx;
+
+    private final Map<UUID, IDataReceiveListenerFactory> pendingConnectionReceivers;
+
+    private final ConnectionListenerThread connectionListenerThread;
+
+    private final DataListenerThread dataListenerThread;
+
+    private final IDataReceiveListener initialDataReceiveListener;
+
+    private volatile boolean stopped;
+
+    private ByteBuffer emptyFrame;
+
+    public ConnectionManager(HyracksContext ctx, InetAddress address) throws IOException {
+        this.ctx = ctx;
+        serverSocketChannel = ServerSocketChannel.open();
+        ServerSocket serverSocket = serverSocketChannel.socket();
+        serverSocket.bind(new InetSocketAddress(address, 0));
+
+        networkAddress = new NetworkAddress(serverSocket.getInetAddress(), serverSocket.getLocalPort());
+
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Connection manager listening on " + serverSocket.getInetAddress() + ":"
+                    + serverSocket.getLocalPort());
+        }
+
+        pendingConnectionReceivers = new HashMap<UUID, IDataReceiveListenerFactory>();
+        dataListenerThread = new DataListenerThread();
+        connectionListenerThread = new ConnectionListenerThread();
+        initialDataReceiveListener = new InitialDataReceiveListener();
+        emptyFrame = ctx.getResourceManager().allocateFrame();
+        emptyFrame.putInt(FrameHelper.getTupleCountOffset(ctx), 0);
+    }
+
+    public synchronized void dumpStats() {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Number of pendingConnectionReceivers: " + pendingConnectionReceivers.size());
+            LOGGER.info("Number of selectable keys: " + dataListenerThread.selector.keys().size());
+        }
+    }
+
+    public NetworkAddress getNetworkAddress() {
+        return networkAddress;
+    }
+
+    public void start() {
+        stopped = false;
+        connectionListenerThread.start();
+        dataListenerThread.start();
+    }
+
+    public void stop() {
+        try {
+            stopped = true;
+            serverSocketChannel.close();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+
+    public IFrameWriter connect(NetworkAddress address, UUID id, int senderId) throws HyracksDataException {
+        try {
+            SocketChannel channel = SocketChannel
+                    .open(new InetSocketAddress(address.getIpAddress(), address.getPort()));
+            byte[] initialFrame = new byte[INITIAL_MESSAGE_LEN];
+            ByteBuffer buffer = ByteBuffer.wrap(initialFrame);
+            buffer.clear();
+            buffer.putLong(id.getMostSignificantBits());
+            buffer.putLong(id.getLeastSignificantBits());
+            buffer.putInt(senderId);
+            buffer.flip();
+            int bytesWritten = 0;
+            while (bytesWritten < INITIAL_MESSAGE_LEN) {
+                int n = channel.write(buffer);
+                if (n < 0) {
+                    throw new HyracksDataException("Stream closed prematurely");
+                }
+                bytesWritten += n;
+            }
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Send Initial message: " + id + ":" + senderId);
+            }
+            buffer.clear();
+            buffer.limit(FrameConstants.SIZE_LEN);
+            int bytesRead = 0;
+            while (bytesRead < FrameConstants.SIZE_LEN) {
+                int n = channel.read(buffer);
+                if (n < 0) {
+                    throw new HyracksDataException("Stream closed prematurely");
+                }
+                bytesRead += n;
+            }
+            buffer.flip();
+            int frameLen = buffer.getInt();
+            if (frameLen != FrameConstants.SIZE_LEN) {
+                throw new IllegalStateException("Received illegal framelen = " + frameLen);
+            }
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Got Ack message: " + id + ":" + senderId);
+            }
+            return new NetworkFrameWriter(channel);
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public synchronized void acceptConnection(UUID id, IDataReceiveListenerFactory receiver) {
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.info("Connection manager accepting " + id);
+        }
+        pendingConnectionReceivers.put(id, receiver);
+    }
+
+    public synchronized void unacceptConnection(UUID id) {
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.info("Connection manager unaccepting " + id);
+        }
+        pendingConnectionReceivers.remove(id);
+    }
+
+    private final class NetworkFrameWriter implements IFrameWriter {
+        private SocketChannel channel;
+
+        NetworkFrameWriter(SocketChannel channel) {
+            this.channel = channel;
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            try {
+                synchronized (emptyFrame) {
+                    emptyFrame.position(0);
+                    emptyFrame.limit(emptyFrame.capacity());
+                    channel.write(emptyFrame);
+                }
+                channel.close();
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            try {
+                if (LOGGER.isLoggable(Level.FINER)) {
+                    int frameLen = buffer.getInt(buffer.position());
+                    LOGGER.finer("ConnectionManager.NetworkFrameWriter: frameLen = " + frameLen);
+                }
+                while (buffer.remaining() > 0) {
+                    channel.write(buffer);
+                }
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+        }
+    }
+
+    private final class ConnectionListenerThread extends Thread {
+        public ConnectionListenerThread() {
+            super("Hyracks Connection Listener Thread");
+        }
+
+        @Override
+        public void run() {
+            while (!stopped) {
+                try {
+                    SocketChannel sc = serverSocketChannel.accept();
+                    dataListenerThread.addSocketChannel(sc);
+                } catch (AsynchronousCloseException e) {
+                    // do nothing
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+
+    private final class DataListenerThread extends Thread {
+        private Selector selector;
+
+        private List<SocketChannel> pendingSockets;
+
+        public DataListenerThread() {
+            super("Hyracks Data Listener Thread");
+            try {
+                selector = Selector.open();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+            pendingSockets = new ArrayList<SocketChannel>();
+        }
+
+        synchronized void addSocketChannel(SocketChannel sc) throws IOException {
+            LOGGER.info("Connection received");
+            pendingSockets.add(sc);
+            selector.wakeup();
+        }
+
+        @Override
+        public void run() {
+            while (!stopped) {
+                try {
+                    if (LOGGER.isLoggable(Level.FINE)) {
+                        LOGGER.fine("Starting Select");
+                    }
+                    int n = selector.select();
+                    synchronized (this) {
+                        if (!pendingSockets.isEmpty()) {
+                            for (SocketChannel sc : pendingSockets) {
+                                sc.configureBlocking(false);
+                                SelectionKey scKey = sc.register(selector, SelectionKey.OP_READ);
+                                ConnectionEntry entry = new ConnectionEntry(ctx, sc, scKey);
+                                entry.setDataReceiveListener(initialDataReceiveListener);
+                                scKey.attach(entry);
+                                if (LOGGER.isLoggable(Level.FINE)) {
+                                    LOGGER.fine("Woke up selector");
+                                }
+                            }
+                            pendingSockets.clear();
+                        }
+                        if (LOGGER.isLoggable(Level.FINE)) {
+                            LOGGER.fine("Selector: " + n);
+                        }
+                        if (n > 0) {
+                            for (Iterator<SelectionKey> i = selector.selectedKeys().iterator(); i.hasNext();) {
+                                SelectionKey key = i.next();
+                                i.remove();
+                                ConnectionEntry entry = (ConnectionEntry) key.attachment();
+                                boolean close = false;
+                                try {
+                                    close = entry.dispatch(key);
+                                } catch (IOException e) {
+                                    e.printStackTrace();
+                                    close = true;
+                                }
+                                if (close) {
+                                    key.cancel();
+                                    entry.close();
+                                }
+                            }
+                        }
+                    }
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+
+    private class InitialDataReceiveListener implements IDataReceiveListener {
+        @Override
+        public void dataReceived(IConnectionEntry entry) throws IOException {
+            ByteBuffer buffer = entry.getReadBuffer();
+            buffer.flip();
+            IDataReceiveListener newListener = null;
+            if (buffer.remaining() >= INITIAL_MESSAGE_LEN) {
+                long msb = buffer.getLong();
+                long lsb = buffer.getLong();
+                UUID endpointID = new UUID(msb, lsb);
+                int senderId = buffer.getInt();
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    LOGGER.fine("Initial Frame received: " + endpointID + ":" + senderId);
+                }
+                IDataReceiveListenerFactory connectionReceiver;
+                synchronized (ConnectionManager.this) {
+                    connectionReceiver = pendingConnectionReceivers.get(endpointID);
+                    if (connectionReceiver == null) {
+                        entry.close();
+                        return;
+                    }
+                }
+
+                newListener = connectionReceiver.getDataReceiveListener(endpointID, entry, senderId);
+                entry.setDataReceiveListener(newListener);
+                byte[] ack = new byte[4];
+                ByteBuffer ackBuffer = ByteBuffer.wrap(ack);
+                ackBuffer.clear();
+                ackBuffer.putInt(FrameConstants.SIZE_LEN);
+                ackBuffer.flip();
+                entry.write(ackBuffer);
+            }
+            buffer.compact();
+            if (newListener != null && buffer.remaining() > 0) {
+                newListener.dataReceived(entry);
+            }
+        }
+
+        @Override
+        public void eos(IConnectionEntry entry) {
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/DemuxDataReceiveListenerFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/DemuxDataReceiveListenerFactory.java
new file mode 100644
index 0000000..3eb2280
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/DemuxDataReceiveListenerFactory.java
@@ -0,0 +1,142 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.UUID;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
+import edu.uci.ics.hyracks.api.comm.IDataReceiveListener;
+import edu.uci.ics.hyracks.api.comm.IDataReceiveListenerFactory;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class DemuxDataReceiveListenerFactory implements IDataReceiveListenerFactory, IConnectionDemultiplexer,
+        IDataReceiveListener {
+    private static final Logger LOGGER = Logger.getLogger(DemuxDataReceiveListenerFactory.class.getName());
+
+    private final NonDeterministicFrameReader frameReader;
+    private final HyracksContext ctx;
+    private final BitSet readyBits;
+    private IConnectionEntry senders[];
+    private int openSenderCount;
+
+    public DemuxDataReceiveListenerFactory(HyracksContext ctx) {
+        frameReader = new NonDeterministicFrameReader(ctx, this);
+        this.ctx = ctx;
+        readyBits = new BitSet();
+        senders = null;
+        openSenderCount = 0;
+    }
+
+    @Override
+    public IDataReceiveListener getDataReceiveListener(UUID endpointUUID, IConnectionEntry entry, int senderIndex) {
+        entry.attach(senderIndex);
+        addSender(senderIndex, entry);
+        return this;
+    }
+
+    public IFrameReader getFrameReader() {
+        return frameReader;
+    }
+
+    @Override
+    public synchronized void dataReceived(IConnectionEntry entry) throws IOException {
+        int senderIndex = (Integer) entry.getAttachment();
+        ByteBuffer buffer = entry.getReadBuffer();
+        buffer.flip();
+        int dataLen = buffer.remaining();
+        if (dataLen >= ctx.getFrameSize()) {
+            if (LOGGER.isLoggable(Level.FINEST)) {
+                LOGGER.finest("NonDeterministicDataReceiveListener: frame received: sender = " + senderIndex);
+            }
+            SelectionKey key = entry.getSelectionKey();
+            int ops = key.interestOps();
+            key.interestOps(ops & ~SelectionKey.OP_READ);
+            readyBits.set(senderIndex);
+            notifyAll();
+            return;
+        }
+        buffer.compact();
+    }
+
+    @Override
+    public void eos(IConnectionEntry entry) {
+    }
+
+    private synchronized void addSender(int senderIndex, IConnectionEntry entry) {
+        readyBits.clear(senderIndex);
+        if (senders == null) {
+            senders = new IConnectionEntry[senderIndex + 1];
+        } else if (senders.length <= senderIndex) {
+            senders = Arrays.copyOf(senders, senderIndex + 1);
+        }
+        senders[senderIndex] = entry;
+        ++openSenderCount;
+    }
+
+    @Override
+    public synchronized IConnectionEntry findNextReadyEntry(int lastReadSender) {
+        while (openSenderCount > 0 && readyBits.isEmpty()) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+            }
+        }
+        lastReadSender = readyBits.nextSetBit(lastReadSender);
+        if (lastReadSender < 0) {
+            lastReadSender = readyBits.nextSetBit(0);
+        }
+        return senders[lastReadSender];
+    }
+
+    @Override
+    public synchronized void unreadyEntry(int index) {
+        readyBits.clear(index);
+        IConnectionEntry entry = senders[index];
+        SelectionKey key = entry.getSelectionKey();
+        if (key.isValid()) {
+            int ops = key.interestOps();
+            key.interestOps(ops | SelectionKey.OP_READ);
+            key.selector().wakeup();
+        }
+    }
+
+    @Override
+    public synchronized int closeEntry(int index) throws HyracksDataException {
+        IConnectionEntry entry = senders[index];
+        SelectionKey key = entry.getSelectionKey();
+        key.cancel();
+        try {
+            entry.close();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+        return --openSenderCount;
+    }
+
+    @Override
+    public synchronized int getSenderCount() {
+        return senders.length;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/IReceiverProtocolStack.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/IReceiverProtocolStack.java
new file mode 100644
index 0000000..b7effd5
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/IReceiverProtocolStack.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;
+
+import edu.uci.ics.hyracks.api.comm.IDataReceiveListenerFactory;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+
+public interface IReceiverProtocolStack {
+    public IDataReceiveListenerFactory createDataReceiveListenerFactory();
+
+    public IFrameReader createFrameReader();
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/ISenderProtocolStack.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/ISenderProtocolStack.java
new file mode 100644
index 0000000..c94c955
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/ISenderProtocolStack.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.comm;
+
+import edu.uci.ics.hyracks.api.comm.Endpoint;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ISenderProtocolStack {
+    public IFrameWriter createFrameWriter(Endpoint endpoint) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/NonDeterministicFrameReader.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/NonDeterministicFrameReader.java
new file mode 100644
index 0000000..3f2d449
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/NonDeterministicFrameReader.java
@@ -0,0 +1,84 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameHelper;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class NonDeterministicFrameReader implements IFrameReader {
+    private static final Logger LOGGER = Logger.getLogger(NonDeterministicFrameReader.class.getName());
+
+    private final HyracksContext ctx;
+    private final IConnectionDemultiplexer demux;
+    private int lastReadSender;
+    private boolean eos;
+
+    public NonDeterministicFrameReader(HyracksContext ctx, IConnectionDemultiplexer demux) {
+        this.ctx = ctx;
+        this.demux = demux;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        lastReadSender = 0;
+        eos = false;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+    }
+
+    @Override
+    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        if (eos) {
+            return false;
+        }
+        while (true) {
+            IConnectionEntry entry = demux.findNextReadyEntry(lastReadSender);
+            lastReadSender = (Integer) entry.getAttachment();
+            ByteBuffer netBuffer = entry.getReadBuffer();
+            int tupleCount = netBuffer.getInt(FrameHelper.getTupleCountOffset(ctx));
+            if (LOGGER.isLoggable(Level.FINER)) {
+                LOGGER.finer("Frame Tuple Count: " + tupleCount);
+            }
+            if (tupleCount == 0) {
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    LOGGER.fine("Empty Frame received: Closing " + lastReadSender);
+                }
+                int openEntries = demux.closeEntry(lastReadSender);
+                if (openEntries == 0) {
+                    eos = true;
+                    return false;
+                }
+                netBuffer.clear();
+                demux.unreadyEntry(lastReadSender);
+            } else {
+                buffer.clear();
+                buffer.put(netBuffer);
+                netBuffer.clear();
+                demux.unreadyEntry(lastReadSender);
+                return true;
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/SortMergeFrameReader.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/SortMergeFrameReader.java
new file mode 100644
index 0000000..62372dc
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/SortMergeFrameReader.java
@@ -0,0 +1,247 @@
+/*
+ * 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;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.Comparator;
+import java.util.PriorityQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameHelper;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.comm.io.FrameTuplePairComparator;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class SortMergeFrameReader implements IFrameReader {
+    private static final Logger LOGGER = Logger.getLogger(SortMergeFrameReader.class.getName());
+
+    private final HyracksContext ctx;
+    private final IConnectionDemultiplexer demux;
+    private final FrameTuplePairComparator tpc;
+    private final FrameTupleAppender appender;
+    private final RecordDescriptor recordDescriptor;
+    private Run[] runs;
+    private ByteBuffer[] frames;
+    private PriorityQueue<Integer> pQueue;
+    private int lastReadSender;
+    private boolean first;
+
+    public SortMergeFrameReader(HyracksContext ctx, IConnectionDemultiplexer demux, int[] sortFields,
+            IBinaryComparator[] comparators, RecordDescriptor recordDescriptor) {
+        this.ctx = ctx;
+        this.demux = demux;
+        tpc = new FrameTuplePairComparator(sortFields, sortFields, comparators);
+        appender = new FrameTupleAppender(ctx);
+        this.recordDescriptor = recordDescriptor;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        int nSenders = demux.getSenderCount();
+        runs = new Run[nSenders];
+        frames = new ByteBuffer[nSenders];
+        for (int i = 0; i < runs.length; ++i) {
+            runs[i] = new Run(i);
+            frames[i] = ctx.getResourceManager().allocateFrame();
+        }
+        pQueue = new PriorityQueue<Integer>(nSenders, new Comparator<Integer>() {
+            @Override
+            public int compare(Integer o1, Integer o2) {
+                int i1 = o1.intValue();
+                int i2 = o2.intValue();
+                Run r1 = runs[i1];
+                Run r2 = runs[i2];
+
+                int c = tpc.compare(r1.accessor, r1.tIndex, r2.accessor, r2.tIndex);
+                return c == 0 ? (i1 < i2 ? -1 : 1) : c;
+            }
+        });
+        lastReadSender = 0;
+        first = true;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        for (Run r : runs) {
+            r.close();
+        }
+    }
+
+    @Override
+    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        buffer.clear();
+        buffer.position(buffer.capacity());
+        appender.reset(buffer, true);
+        if (first) {
+            for (int i = 0; i < runs.length; ++i) {
+                if (runs[i].next()) {
+                    pQueue.add(Integer.valueOf(i));
+                }
+            }
+        }
+        first = false;
+        while (true) {
+            if (pQueue.isEmpty()) {
+                return appender.getTupleCount() > 0;
+            }
+            Integer top = pQueue.peek();
+            Run run = runs[top.intValue()];
+            if (!appender.append(run.accessor, run.tIndex)) {
+                return true;
+            }
+            pQueue.remove();
+            if (run.next()) {
+                pQueue.add(top);
+            }
+        }
+    }
+
+    private class Run {
+        private final int runId;
+        private final File file;
+        private final FileChannel channel;
+        private final ByteBuffer frame;
+        private final FrameTupleAccessor accessor;
+        private int tIndex;
+        private long readFP;
+        private long writeFP;
+        private boolean eof;
+
+        public Run(int runId) throws HyracksDataException {
+            this.runId = runId;
+            try {
+                file = ctx.getResourceManager().createFile(SortMergeFrameReader.class.getSimpleName(), ".run");
+                RandomAccessFile raf = new RandomAccessFile(file, "rw");
+                channel = raf.getChannel();
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+            frame = ctx.getResourceManager().allocateFrame();
+            accessor = new FrameTupleAccessor(ctx, recordDescriptor);
+            readFP = 0;
+            writeFP = 0;
+            eof = false;
+        }
+
+        public void close() throws HyracksDataException {
+            try {
+                channel.close();
+                file.delete();
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        private void write(ByteBuffer frame) throws HyracksDataException {
+            try {
+                int len = frame.capacity();
+                while (len > 0) {
+                    int sz = channel.write(frame, writeFP);
+                    if (sz < 0) {
+                        throw new HyracksDataException("Error writing to run");
+                    }
+                    len -= sz;
+                    writeFP += sz;
+                }
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        private boolean next() throws HyracksDataException {
+            ++tIndex;
+            while (readFP == 0 || tIndex >= accessor.getTupleCount()) {
+                if (!read(frame)) {
+                    return false;
+                }
+                accessor.reset(frame);
+                tIndex = 0;
+            }
+            return true;
+        }
+
+        private boolean read(ByteBuffer frame) throws HyracksDataException {
+            while (!eof && readFP >= writeFP) {
+                spoolRuns(runId);
+            }
+            if (eof && readFP >= writeFP) {
+                return false;
+            }
+            try {
+                channel.position(readFP);
+                frame.clear();
+                int len = frame.capacity();
+                while (len > 0) {
+                    int sz = channel.read(frame, readFP);
+                    if (sz < 0) {
+                        throw new HyracksDataException("Error reading file");
+                    }
+                    len -= sz;
+                    readFP += sz;
+                }
+                return true;
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        private void eof() {
+            eof = true;
+        }
+    }
+
+    private void spoolRuns(int interestingRun) throws HyracksDataException {
+        while (true) {
+            IConnectionEntry entry = demux.findNextReadyEntry(lastReadSender);
+            lastReadSender = (Integer) entry.getAttachment();
+            ByteBuffer netBuffer = entry.getReadBuffer();
+            int tupleCount = netBuffer.getInt(FrameHelper.getTupleCountOffset(ctx));
+            if (LOGGER.isLoggable(Level.FINER)) {
+                LOGGER.finer("Frame Tuple Count: " + tupleCount);
+            }
+            if (tupleCount == 0) {
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    LOGGER.fine("Empty Frame received: Closing " + lastReadSender);
+                }
+                int openEntries = demux.closeEntry(lastReadSender);
+                runs[lastReadSender].eof();
+                netBuffer.clear();
+                demux.unreadyEntry(lastReadSender);
+                if (openEntries == 0) {
+                    return;
+                }
+            } else {
+                runs[lastReadSender].write(netBuffer);
+                netBuffer.clear();
+                demux.unreadyEntry(lastReadSender);
+            }
+            if (lastReadSender == interestingRun) {
+                return;
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/ArrayTupleBuilder.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/ArrayTupleBuilder.java
new file mode 100644
index 0000000..d800fba
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/ArrayTupleBuilder.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.comm.io;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ArrayTupleBuilder {
+    private final ByteArrayAccessibleOutputStream baaos;
+    private final DataOutputStream dos;
+    private final int[] fEndOffsets;
+    private int nextField;
+
+    public ArrayTupleBuilder(int nFields) {
+        baaos = new ByteArrayAccessibleOutputStream();
+        dos = new DataOutputStream(baaos);
+        fEndOffsets = new int[nFields];
+    }
+
+    public void reset() {
+        nextField = 0;
+        baaos.reset();
+    }
+
+    public int[] getFieldEndOffsets() {
+        return fEndOffsets;
+    }
+
+    public byte[] getByteArray() {
+        return baaos.getByteArray();
+    }
+
+    public int getSize() {
+        return baaos.size();
+    }
+
+    public void addField(FrameTupleAccessor accessor, int tIndex, int fIndex) throws HyracksDataException {
+        int startOffset = accessor.getTupleStartOffset(tIndex);
+        int fStartOffset = accessor.getFieldStartOffset(tIndex, fIndex);
+        int fLen = accessor.getFieldEndOffset(tIndex, fIndex) - fStartOffset;
+        try {
+            dos.write(accessor.getBuffer().array(), startOffset + accessor.getFieldSlotsLength() + fStartOffset, fLen);
+            if (FrameConstants.DEBUG_FRAME_IO) {
+                dos.writeInt(FrameConstants.FRAME_FIELD_MAGIC);
+            }
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+        fEndOffsets[nextField++] = baaos.size();
+    }
+
+    public <T> void addField(ISerializerDeserializer<T> serDeser, T instance) throws HyracksDataException {
+        serDeser.serialize(instance, dos);
+        fEndOffsets[nextField++] = baaos.size();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/ByteArrayAccessibleOutputStream.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/ByteArrayAccessibleOutputStream.java
new file mode 100644
index 0000000..41c771b
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/ByteArrayAccessibleOutputStream.java
@@ -0,0 +1,42 @@
+/*
+ * 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.io;
+
+import java.io.ByteArrayOutputStream;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class ByteArrayAccessibleOutputStream extends ByteArrayOutputStream {
+    private static final Logger LOGGER = Logger.getLogger(ByteArrayAccessibleOutputStream.class.getName());
+
+    public byte[] getByteArray() {
+        return buf;
+    }
+
+    public void write(int b) {
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("write(byte) value: " + b);
+        }
+        super.write(b);
+    }
+
+    @Override
+    public void write(byte[] bytes, int offset, int length) {
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("write(byte[], int, int) offset: " + offset + " length" + length);
+        }
+        super.write(bytes, offset, length);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameDeserializer.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameDeserializer.java
new file mode 100644
index 0000000..22d4961
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameDeserializer.java
@@ -0,0 +1,98 @@
+/*
+ * 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.io;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.util.ByteBufferInputStream;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class FrameDeserializer {
+    private static final Logger LOGGER = Logger.getLogger(FrameDeserializer.class.getName());
+
+    private final ByteBufferInputStream bbis;
+
+    private final DataInputStream di;
+
+    private final RecordDescriptor recordDescriptor;
+
+    private final FrameTupleAccessor frameTupleAccessor;
+
+    private int tupleCount;
+
+    private int tIndex;
+
+    private ByteBuffer buffer;
+
+    public FrameDeserializer(HyracksContext ctx, RecordDescriptor recordDescriptor) {
+        this.bbis = new ByteBufferInputStream();
+        this.di = new DataInputStream(bbis);
+        this.recordDescriptor = recordDescriptor;
+        frameTupleAccessor = new FrameTupleAccessor(ctx, recordDescriptor);
+    }
+
+    public void reset(ByteBuffer buffer) {
+        this.buffer = buffer;
+        frameTupleAccessor.reset(buffer);
+        tupleCount = frameTupleAccessor.getTupleCount();
+        tIndex = 0;
+    }
+
+    public boolean done() {
+        return tIndex >= tupleCount;
+    }
+
+    public Object[] deserializeRecord() throws HyracksDataException {
+        int start = frameTupleAccessor.getTupleStartOffset(tIndex) + frameTupleAccessor.getFieldSlotsLength();
+        bbis.setByteBuffer(buffer, start);
+        Object[] record = new Object[recordDescriptor.getFields().length];
+        for (int i = 0; i < record.length; ++i) {
+            Object instance = recordDescriptor.getFields()[i].deserialize(di);
+            if (LOGGER.isLoggable(Level.FINEST)) {
+                LOGGER.finest(i + " " + instance);
+            }
+            record[i] = instance;
+            if (FrameConstants.DEBUG_FRAME_IO) {
+                try {
+                    if (di.readInt() != FrameConstants.FRAME_FIELD_MAGIC) {
+                        throw new HyracksDataException("Field magic mismatch");
+                    }
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("Read Record tIndex = " + tIndex + ", tupleCount = " + tupleCount);
+        }
+        ++tIndex;
+        return record;
+    }
+
+    public void close() {
+        try {
+            di.close();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameDeserializingDataReader.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameDeserializingDataReader.java
new file mode 100644
index 0000000..31d37b7
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameDeserializingDataReader.java
@@ -0,0 +1,75 @@
+/*
+ * 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.io;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class FrameDeserializingDataReader implements IOpenableDataReader<Object[]> {
+    private final ByteBuffer buffer;
+
+    private boolean eos;
+
+    private boolean first;
+
+    private final IFrameReader frameReader;
+
+    private final FrameDeserializer frameDeserializer;
+
+    public FrameDeserializingDataReader(HyracksContext ctx, IFrameReader frameReader, RecordDescriptor recordDescriptor) {
+        buffer = ctx.getResourceManager().allocateFrame();
+        this.frameReader = frameReader;
+        this.frameDeserializer = new FrameDeserializer(ctx, recordDescriptor);
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        frameReader.open();
+        buffer.clear();
+        buffer.flip();
+        eos = false;
+        first = true;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        frameReader.close();
+        frameDeserializer.close();
+    }
+
+    @Override
+    public Object[] readData() throws HyracksDataException {
+        while (true) {
+            if (eos) {
+                return null;
+            }
+            if (!first && !frameDeserializer.done()) {
+                return frameDeserializer.deserializeRecord();
+            }
+            buffer.clear();
+            if (!frameReader.nextFrame(buffer)) {
+                eos = true;
+            } else {
+                frameDeserializer.reset(buffer);
+            }
+            first = false;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameDeserializingDataWriter.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameDeserializingDataWriter.java
new file mode 100644
index 0000000..e0a7250
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameDeserializingDataWriter.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.comm.io;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class FrameDeserializingDataWriter implements IFrameWriter {
+    private final IOpenableDataWriter<Object[]> writer;
+    private final FrameDeserializer frameDeserializer;
+
+    public FrameDeserializingDataWriter(HyracksContext ctx, IOpenableDataWriter<Object[]> writer,
+            RecordDescriptor recordDescriptor) {
+        this.writer = writer;
+        this.frameDeserializer = new FrameDeserializer(ctx, recordDescriptor);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        writer.close();
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        frameDeserializer.reset(buffer);
+        while (!frameDeserializer.done()) {
+            Object[] tuple = frameDeserializer.deserializeRecord();
+            writer.writeData(tuple);
+        }
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        writer.open();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameHelper.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameHelper.java
new file mode 100644
index 0000000..a82a051
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameHelper.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.comm.io;
+
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class FrameHelper {
+    public static int getTupleCountOffset(HyracksContext ctx) {
+        return ctx.getFrameSize() - 4;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameTupleAccessor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameTupleAccessor.java
new file mode 100644
index 0000000..05d83cd
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameTupleAccessor.java
@@ -0,0 +1,99 @@
+/*
+ * 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.io;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.util.ByteBufferInputStream;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+/**
+ * FrameTupleCursor is used to navigate over tuples in a Frame.
+ * A frame is formatted with tuple data concatenated starting at offset 0, one tuple after another.
+ * Offset FS - 4 holds an int indicating the number of tuples (N) in the frame. FS - ((i + 1) * 4) for i from
+ * 0 to N - 1 holds an int indicating the offset of the (i + 1)^th tuple.
+ * Every tuple is organized as a sequence of shorts indicating the end of each field in the tuple relative to the end of the
+ * field slots.
+ * 
+ * @author vinayakb
+ */
+public final class FrameTupleAccessor {
+    private final HyracksContext ctx;
+    private final RecordDescriptor recordDescriptor;
+
+    private ByteBuffer buffer;
+
+    public FrameTupleAccessor(HyracksContext ctx, RecordDescriptor recordDescriptor) {
+        this.ctx = ctx;
+        this.recordDescriptor = recordDescriptor;
+    }
+
+    public void reset(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    public int getTupleCount() {
+        return buffer.getInt(FrameHelper.getTupleCountOffset(ctx));
+    }
+
+    public int getTupleStartOffset(int tupleIndex) {
+        return tupleIndex == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(ctx) - 4 * tupleIndex);
+    }
+
+    public int getTupleEndOffset(int tupleIndex) {
+        return buffer.getInt(FrameHelper.getTupleCountOffset(ctx) - 4 * (tupleIndex + 1));
+    }
+
+    public int getFieldStartOffset(int tupleIndex, int fIdx) {
+        return fIdx == 0 ? 0 : buffer.getShort(getTupleStartOffset(tupleIndex) + (fIdx - 1) * 2);
+    }
+
+    public int getFieldEndOffset(int tupleIndex, int fIdx) {
+        return buffer.getShort(getTupleStartOffset(tupleIndex) + fIdx * 2);
+    }
+
+    public int getFieldSlotsLength() {
+        return recordDescriptor.getFields().length * 2;
+    }
+
+    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) + ")[");
+            for (int j = 0; j < recordDescriptor.getFields().length; ++j) {
+                System.err.print(j + ":(" + getFieldStartOffset(i, j) + ", " + getFieldEndOffset(i, j) + ") ");
+                System.err.print("{");
+                bbis.setByteBuffer(buffer, getTupleStartOffset(i) + getFieldSlotsLength() + getFieldStartOffset(i, j));
+                try {
+                    System.err.print(recordDescriptor.getFields()[j].deserialize(dis));
+                } catch (HyracksDataException e) {
+                    e.printStackTrace();
+                }
+                System.err.print("}");
+            }
+            System.err.println("]");
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameTupleAppender.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameTupleAppender.java
new file mode 100644
index 0000000..2ae094e
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameTupleAppender.java
@@ -0,0 +1,155 @@
+/*
+ * 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.io;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class FrameTupleAppender {
+    private final HyracksContext ctx;
+
+    private ByteBuffer buffer;
+
+    private int tupleCount;
+
+    private int tupleDataEndOffset;
+
+    public FrameTupleAppender(HyracksContext ctx) {
+        this.ctx = ctx;
+    }
+
+    public void reset(ByteBuffer buffer, boolean clear) {
+        this.buffer = buffer;
+        if (clear) {
+            buffer.putInt(FrameHelper.getTupleCountOffset(ctx), 0);
+            tupleCount = 0;
+            tupleDataEndOffset = 0;
+        } else {
+            tupleCount = buffer.getInt(FrameHelper.getTupleCountOffset(ctx));
+            tupleDataEndOffset = tupleCount == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(ctx) - tupleCount
+                    * 4);
+        }
+    }
+
+    public boolean append(int[] fieldSlots, byte[] bytes, int offset, int length) {
+        if (tupleDataEndOffset + fieldSlots.length * 2 + length + 4 + (tupleCount + 1) * 4 <= ctx.getFrameSize()) {
+            for (int i = 0; i < fieldSlots.length; ++i) {
+                buffer.putShort(tupleDataEndOffset + i * 2, (short) fieldSlots[i]);
+            }
+            System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset + fieldSlots.length * 2, length);
+            tupleDataEndOffset += fieldSlots.length * 2 + length;
+            buffer.putInt(FrameHelper.getTupleCountOffset(ctx) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            ++tupleCount;
+            buffer.putInt(FrameHelper.getTupleCountOffset(ctx), tupleCount);
+            return true;
+        }
+        return false;
+    }
+
+    public boolean append(FrameTupleAccessor tupleAccessor, int tIndex) {
+        int startOffset = tupleAccessor.getTupleStartOffset(tIndex);
+        int endOffset = tupleAccessor.getTupleEndOffset(tIndex);
+        int length = endOffset - startOffset;
+        if (tupleDataEndOffset + length + 4 + (tupleCount + 1) * 4 <= ctx.getFrameSize()) {
+            ByteBuffer src = tupleAccessor.getBuffer();
+            System.arraycopy(src.array(), startOffset, buffer.array(), tupleDataEndOffset, length);
+            tupleDataEndOffset += length;
+            buffer.putInt(FrameHelper.getTupleCountOffset(ctx) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            ++tupleCount;
+            buffer.putInt(FrameHelper.getTupleCountOffset(ctx), tupleCount);
+            return true;
+        }
+        return false;
+    }
+
+    public boolean appendConcat(FrameTupleAccessor accessor0, int tIndex0, FrameTupleAccessor accessor1, int tIndex1) {
+        int startOffset0 = accessor0.getTupleStartOffset(tIndex0);
+        int endOffset0 = accessor0.getTupleEndOffset(tIndex0);
+        int length0 = endOffset0 - startOffset0;
+
+        int startOffset1 = accessor1.getTupleStartOffset(tIndex1);
+        int endOffset1 = accessor1.getTupleEndOffset(tIndex1);
+        int length1 = endOffset1 - startOffset1;
+
+        if (tupleDataEndOffset + length0 + length1 + 4 + (tupleCount + 1) * 4 <= ctx.getFrameSize()) {
+            ByteBuffer src0 = accessor0.getBuffer();
+            ByteBuffer src1 = accessor1.getBuffer();
+            int slotsLen0 = accessor0.getFieldSlotsLength();
+            int slotsLen1 = accessor1.getFieldSlotsLength();
+            int dataLen0 = length0 - slotsLen0;
+            int dataLen1 = length1 - slotsLen1;
+            // Copy slots from accessor0 verbatim
+            System.arraycopy(src0.array(), startOffset0, buffer.array(), tupleDataEndOffset, slotsLen0);
+            // Copy slots from accessor1 with the following transformation: newSlotIdx = oldSlotIdx + dataLen0
+            for (int i = 0; i < slotsLen1 / 2; ++i) {
+                buffer.putShort(tupleDataEndOffset + slotsLen0 + i * 2,
+                        (short) (src1.getShort(startOffset1 + i * 2) + dataLen0));
+            }
+            // Copy data0
+            System.arraycopy(src0.array(), startOffset0 + slotsLen0, buffer.array(), tupleDataEndOffset + slotsLen0
+                    + slotsLen1, dataLen0);
+            // Copy data1
+            System.arraycopy(src1.array(), startOffset1 + slotsLen1, buffer.array(), tupleDataEndOffset + slotsLen0
+                    + slotsLen1 + dataLen0, dataLen1);
+            tupleDataEndOffset += (length0 + length1);
+            buffer.putInt(FrameHelper.getTupleCountOffset(ctx) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            ++tupleCount;
+            buffer.putInt(FrameHelper.getTupleCountOffset(ctx), tupleCount);
+            return true;
+        }
+        return false;
+    }
+
+    public boolean appendProjection(FrameTupleAccessor accessor, int tIndex, int[] fields) {
+        int fTargetSlotsLength = fields.length * 2;
+        int length = fTargetSlotsLength;
+        for (int i = 0; i < fields.length; ++i) {
+            length += (accessor.getFieldEndOffset(tIndex, fields[i]) - accessor.getFieldStartOffset(tIndex, fields[i]));
+        }
+
+        if (tupleDataEndOffset + length + 4 + (tupleCount + 1) * 4 <= ctx.getFrameSize()) {
+            int fSrcSlotsLength = accessor.getFieldSlotsLength();
+            int tStartOffset = accessor.getTupleStartOffset(tIndex);
+
+            int fStartOffset = 0;
+            int fEndOffset = 0;
+            for (int i = 0; i < fields.length; ++i) {
+                int fSrcStart = tStartOffset + fSrcSlotsLength + accessor.getFieldStartOffset(tIndex, fields[i]);
+                int fLen = accessor.getFieldEndOffset(tIndex, fields[i])
+                        - accessor.getFieldStartOffset(tIndex, fields[i]);
+                System.arraycopy(accessor.getBuffer().array(), fSrcStart, buffer.array(), tupleDataEndOffset
+                        + fSrcSlotsLength + fStartOffset, fLen);
+                fEndOffset += fLen;
+                buffer.putShort(tupleDataEndOffset + i * 2, (short) fEndOffset);
+                fStartOffset = fEndOffset;
+            }
+            tupleDataEndOffset += length;
+            buffer.putInt(FrameHelper.getTupleCountOffset(ctx) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            ++tupleCount;
+            buffer.putInt(FrameHelper.getTupleCountOffset(ctx), tupleCount);
+            return true;
+        }
+        return false;
+    }
+
+    public int getTupleCount() {
+        return tupleCount;
+    }
+
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameTuplePairComparator.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameTuplePairComparator.java
new file mode 100644
index 0000000..0b23383
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/FrameTuplePairComparator.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.comm.io;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+
+public class FrameTuplePairComparator {
+    private final int[] keys0;
+    private final int[] keys1;
+    private final IBinaryComparator[] comparators;
+
+    public FrameTuplePairComparator(int[] keys0, int[] keys1, IBinaryComparator[] comparators) {
+        this.keys0 = keys0;
+        this.keys1 = keys1;
+        this.comparators = comparators;
+    }
+
+    public int compare(FrameTupleAccessor accessor0, int tIndex0, FrameTupleAccessor accessor1, int tIndex1) {
+        int tStart0 = accessor0.getTupleStartOffset(tIndex0);
+        int fStartOffset0 = accessor0.getFieldSlotsLength() + tStart0;
+
+        int tStart1 = accessor1.getTupleStartOffset(tIndex1);
+        int fStartOffset1 = accessor1.getFieldSlotsLength() + tStart1;
+
+        for (int i = 0; i < keys0.length; ++i) {
+            int fIdx0 = keys0[i];
+            int fStart0 = accessor0.getFieldStartOffset(tIndex0, fIdx0);
+            int fEnd0 = accessor0.getFieldEndOffset(tIndex0, fIdx0);
+            int fLen0 = fEnd0 - fStart0;
+
+            int fIdx1 = keys1[i];
+            int fStart1 = accessor1.getFieldStartOffset(tIndex1, fIdx1);
+            int fEnd1 = accessor1.getFieldEndOffset(tIndex1, fIdx1);
+            int fLen1 = fEnd1 - fStart1;
+
+            int c = comparators[i].compare(accessor0.getBuffer().array(), fStart0 + fStartOffset0, fLen0, accessor1
+                    .getBuffer().array(), fStart1 + fStartOffset1, fLen1);
+            if (c != 0) {
+                return c;
+            }
+        }
+        return 0;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/SerializingDataWriter.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/SerializingDataWriter.java
new file mode 100644
index 0000000..4103dd6
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/io/SerializingDataWriter.java
@@ -0,0 +1,101 @@
+/*
+ * 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.io;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class SerializingDataWriter implements IOpenableDataWriter<Object[]> {
+    private static final Logger LOGGER = Logger.getLogger(SerializingDataWriter.class.getName());
+
+    private final ByteBuffer buffer;
+
+    private final ArrayTupleBuilder tb;
+
+    private final RecordDescriptor recordDescriptor;
+
+    private final IFrameWriter frameWriter;
+
+    private final FrameTupleAppender tupleAppender;
+
+    private boolean open;
+
+    public SerializingDataWriter(HyracksContext ctx, RecordDescriptor recordDescriptor, IFrameWriter frameWriter) {
+        buffer = ctx.getResourceManager().allocateFrame();
+        tb = new ArrayTupleBuilder(recordDescriptor.getFields().length);
+        this.recordDescriptor = recordDescriptor;
+        this.frameWriter = frameWriter;
+        tupleAppender = new FrameTupleAppender(ctx);
+        open = false;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        frameWriter.open();
+        buffer.clear();
+        open = true;
+        tupleAppender.reset(buffer, true);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (!open) {
+            throw new HyracksDataException("Closing SerializingDataWriter that has not been opened");
+        }
+        if (tupleAppender.getTupleCount() > 0) {
+            flushFrame();
+        }
+        frameWriter.close();
+        open = false;
+    }
+
+    @Override
+    public void writeData(Object[] data) throws HyracksDataException {
+        if (!open) {
+            throw new HyracksDataException("Writing to SerializingDataWriter that has not been opened");
+        }
+        tb.reset();
+        for (int i = 0; i < data.length; ++i) {
+            Object instance = data[i];
+            if (LOGGER.isLoggable(Level.FINEST)) {
+                LOGGER.finest(i + " " + instance);
+            }
+            tb.addField(recordDescriptor.getFields()[i], instance);
+        }
+        if (!tupleAppender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+            if (LOGGER.isLoggable(Level.FINEST)) {
+                LOGGER.finest("Flushing: position = " + buffer.position());
+            }
+            flushFrame();
+            tupleAppender.reset(buffer, true);
+            if (!tupleAppender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                throw new IllegalStateException();
+            }
+        }
+    }
+
+    private void flushFrame() throws HyracksDataException {
+        buffer.position(0);
+        buffer.limit(buffer.capacity());
+        frameWriter.nextFrame(buffer);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/util/ByteBufferInputStream.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/util/ByteBufferInputStream.java
new file mode 100644
index 0000000..90f7433
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/util/ByteBufferInputStream.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.comm.util;
+
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public class ByteBufferInputStream extends InputStream {
+    private static final Logger LOGGER = Logger.getLogger(ByteBufferInputStream.class.getName());
+
+    private ByteBuffer buffer;
+
+    private int position;
+
+    public ByteBufferInputStream() {
+    }
+
+    public void setByteBuffer(ByteBuffer buffer, int position) {
+        this.buffer = buffer;
+        this.position = position;
+    }
+
+    @Override
+    public int read() {
+        int remaining = buffer.capacity() - position;
+        int value = remaining > 0 ? (buffer.get(position++) & 0xff) : -1;
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("read(): value: " + value + " remaining: " + remaining + " position: " + position);
+        }
+        return value;
+    }
+
+    @Override
+    public int read(byte[] bytes, int offset, int length) {
+        int remaining = buffer.capacity() - position;
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("read(bytes[], int, int): remaining: " + remaining + " offset: " + offset + " length: "
+                    + length + " position: " + position);
+        }
+        if (remaining == 0) {
+            return -1;
+        }
+        int l = Math.min(length, remaining);
+        System.arraycopy(buffer.array(), position, bytes, offset, l);
+        position += l;
+        return l;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/util/FrameUtils.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/util/FrameUtils.java
new file mode 100644
index 0000000..bdce4a4
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/comm/util/FrameUtils.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.comm.util;
+
+import java.nio.ByteBuffer;
+
+public class FrameUtils {
+    public static void copy(ByteBuffer srcFrame, ByteBuffer destFrame) {
+        makeReadable(srcFrame);
+        destFrame.clear();
+        destFrame.put(srcFrame);
+    }
+
+    public static void makeReadable(ByteBuffer frame) {
+        frame.position(0);
+        frame.limit(frame.capacity());
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/config/CCConfig.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/config/CCConfig.java
new file mode 100644
index 0000000..950569d
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/config/CCConfig.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.config;
+
+import org.kohsuke.args4j.Option;
+
+public class CCConfig {
+    @Option(name = "-port", usage = "Sets the port to listen for connections from node controllers (default 1099)")
+    public int port = 1099;
+
+    @Option(name = "-http-port", usage = "Sets the http port for the admin console")
+    public int httpPort;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/config/NCConfig.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/config/NCConfig.java
new file mode 100644
index 0000000..c95f1ad
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/config/NCConfig.java
@@ -0,0 +1,45 @@
+/*
+ * 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.config;
+
+import java.io.Serializable;
+
+import org.kohsuke.args4j.Option;
+
+public class NCConfig implements Serializable{
+    @Option(name = "-cc-host", usage = "Cluster Controller host name")
+    public String ccHost;
+
+    @Option(name = "-cc-port", usage = "Cluster Controller port (default: 1099)")
+    public int ccPort = 1099;
+
+    @Option(name = "-node-id", usage = "Logical name of node controller unique within the cluster")
+    public String nodeId;
+
+    @Option(name = "-data-ip-address", usage = "IP Address to bind data listener")
+    public String dataIPAddress;
+
+    @Option(name = "-frame-size", usage = "Frame Size to use for data communication (default: 32768)")
+    public int frameSize = 32768;
+
+    @Option(name = "-dcache-client-servers", usage = "Sets the list of DCache servers in the format host1:port1,host2:port2,... (default localhost:54583)")
+    public String dcacheClientServers = "localhost:54583";
+
+    @Option(name = "-dcache-client-server-local", usage = "Sets the local DCache server, if one is available, in the format host:port (default not set)")
+    public String dcacheClientServerLocal;
+
+    @Option(name = "-dcache-client-path", usage = "Sets the path to store the files retrieved from the DCache server (default /tmp/dcache-client)")
+    public String dcacheClientPath = "/tmp/dcache-client";
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/context/HyracksContext.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/context/HyracksContext.java
new file mode 100644
index 0000000..7fe088d
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/context/HyracksContext.java
@@ -0,0 +1,35 @@
+/*
+ * 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.context;
+
+import edu.uci.ics.hyracks.resources.ResourceManager;
+
+public class HyracksContext {
+    private final ResourceManager resourceManager;
+    private final int frameSize;
+
+    public HyracksContext(int frameSize) {
+        resourceManager = new ResourceManager(this);
+        this.frameSize = frameSize;
+    }
+
+    public ResourceManager getResourceManager() {
+        return resourceManager;
+    }
+
+    public int getFrameSize() {
+        return frameSize;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/controller/AbstractRemoteService.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/controller/AbstractRemoteService.java
new file mode 100644
index 0000000..9c8f3dd
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/controller/AbstractRemoteService.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.controller;
+
+import java.rmi.RemoteException;
+import java.rmi.server.UnicastRemoteObject;
+
+import edu.uci.ics.hyracks.service.IService;
+
+public abstract class AbstractRemoteService extends UnicastRemoteObject implements IService {
+    private static final long serialVersionUID = 1L;
+
+    public AbstractRemoteService() throws RemoteException {
+    }
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/controller/ClusterControllerService.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/controller/ClusterControllerService.java
new file mode 100644
index 0000000..0461499
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/controller/ClusterControllerService.java
@@ -0,0 +1,208 @@
+/*
+ * 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.controller;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.rmi.registry.LocateRegistry;
+import java.rmi.registry.Registry;
+import java.util.EnumSet;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.eclipse.jetty.server.Handler;
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.server.handler.AbstractHandler;
+import org.eclipse.jetty.server.handler.ContextHandler;
+
+import edu.uci.ics.hyracks.api.controller.IClusterController;
+import edu.uci.ics.hyracks.api.controller.INodeController;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.api.job.statistics.JobStatistics;
+import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
+import edu.uci.ics.hyracks.config.CCConfig;
+import edu.uci.ics.hyracks.job.JobManager;
+import edu.uci.ics.hyracks.web.WebServer;
+
+public class ClusterControllerService extends AbstractRemoteService implements IClusterController {
+    private static final long serialVersionUID = 1L;
+
+    private CCConfig ccConfig;
+
+    private static Logger LOGGER = Logger.getLogger(ClusterControllerService.class.getName());
+
+    private final Map<String, INodeController> nodeRegistry;
+
+    private WebServer webServer;
+
+    private final JobManager jobManager;
+
+    private final Executor taskExecutor;
+
+    public ClusterControllerService(CCConfig ccConfig) throws Exception {
+        this.ccConfig = ccConfig;
+        nodeRegistry = new LinkedHashMap<String, INodeController>();
+        jobManager = new JobManager(this);
+        taskExecutor = Executors.newCachedThreadPool();
+        webServer = new WebServer(new Handler[] { getAdminConsoleHandler() });
+    }
+
+    @Override
+    public void start() throws Exception {
+        LOGGER.log(Level.INFO, "Starting ClusterControllerService");
+        Registry registry = LocateRegistry.createRegistry(ccConfig.port);
+        registry.rebind(IClusterController.class.getName(), this);
+        webServer.setPort(ccConfig.httpPort);
+        webServer.start();
+        LOGGER.log(Level.INFO, "Started ClusterControllerService");
+    }
+
+    @Override
+    public void stop() throws Exception {
+        LOGGER.log(Level.INFO, "Stopping ClusterControllerService");
+        webServer.stop();
+        LOGGER.log(Level.INFO, "Stopped ClusterControllerService");
+    }
+
+    @Override
+    public UUID createJob(JobSpecification jobSpec) throws Exception {
+        return jobManager.createJob(jobSpec, EnumSet.noneOf(JobFlag.class));
+    }
+
+    @Override
+    public UUID createJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+        return jobManager.createJob(jobSpec, jobFlags);
+    }
+
+    @Override
+    public void registerNode(INodeController nodeController) throws Exception {
+        String id = nodeController.getId();
+        synchronized (this) {
+            if (nodeRegistry.containsKey(id)) {
+                throw new Exception("Node with this name already registered.");
+            }
+            nodeRegistry.put(id, nodeController);
+        }
+        nodeController.notifyRegistration(this);
+        LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
+    }
+
+    @Override
+    public void unregisterNode(INodeController nodeController) throws Exception {
+        String id = nodeController.getId();
+        synchronized (this) {
+            nodeRegistry.remove(id);
+        }
+        LOGGER.log(Level.INFO, "Unregistered INodeController");
+    }
+
+    @Override
+    public INodeController lookupNode(String id) throws Exception {
+        return nodeRegistry.get(id);
+    }
+
+    public Executor getExecutor() {
+        return taskExecutor;
+    }
+
+    public synchronized void notifyJobComplete(final UUID jobId) {
+        for (final INodeController nc : nodeRegistry.values()) {
+            taskExecutor.execute(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        nc.cleanUpJob(jobId);
+                    } catch (Exception e) {
+                    }
+                }
+
+            });
+        }
+    }
+
+    @Override
+    public void notifyStageletComplete(UUID jobId, UUID stageId, String nodeId, StageletStatistics statistics)
+            throws Exception {
+        jobManager.notifyStageletComplete(jobId, stageId, nodeId, statistics);
+    }
+
+    @Override
+    public JobStatus getJobStatus(UUID jobId) throws Exception {
+        return jobManager.getJobStatus(jobId);
+    }
+
+    @Override
+    public void start(UUID jobId) throws Exception {
+        jobManager.start(jobId);
+    }
+
+    @Override
+    public JobStatistics waitForCompletion(UUID jobId) throws Exception {
+        return jobManager.waitForCompletion(jobId);
+    }
+
+    private Handler getAdminConsoleHandler() {
+        ContextHandler handler = new ContextHandler("/admin");
+        handler.setHandler(new AbstractHandler() {
+            @Override
+            public void handle(String target, Request baseRequest, HttpServletRequest request,
+                    HttpServletResponse response) throws IOException, ServletException {
+                if (!"/".equals(target)) {
+                    return;
+                }
+                response.setContentType("text/html;charset=utf-8");
+                response.setStatus(HttpServletResponse.SC_OK);
+                baseRequest.setHandled(true);
+                PrintWriter writer = response.getWriter();
+                writer.println("<html><head><title>Hyracks Admin Console</title></head><body>");
+                writer.println("<h1>Hyracks Admin Console</h1>");
+                writer.println("<h2>Node Controllers</h2>");
+                writer.println("<table><tr><td>Node Id</td><td>Host</td></tr>");
+                synchronized (ClusterControllerService.this) {
+                    for (Map.Entry<String, INodeController> e : nodeRegistry.entrySet()) {
+                        try {
+                            INodeController n = e.getValue();
+                            writer.print("<tr><td>");
+                            writer.print(e.getKey());
+                            writer.print("</td><td>");
+                            writer.print("</td></tr>");
+                        } catch (Exception ex) {
+                        }
+                    }
+                }
+                writer.println("</table>");
+                writer.println("</body></html>");
+                writer.flush();
+            }
+        });
+        return handler;
+    }
+
+    @Override
+    public Map<String, INodeController> getRegistry() throws Exception {
+        return nodeRegistry;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/controller/NodeControllerService.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/controller/NodeControllerService.java
new file mode 100644
index 0000000..dbdf1c9
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/controller/NodeControllerService.java
@@ -0,0 +1,374 @@
+/*
+ * 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.controller;
+
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.rmi.registry.LocateRegistry;
+import java.rmi.registry.Registry;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import edu.uci.ics.hyracks.api.comm.Endpoint;
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.controller.IClusterController;
+import edu.uci.ics.hyracks.api.controller.INodeController;
+import edu.uci.ics.hyracks.api.controller.NodeCapability;
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.OperatorInstanceId;
+import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.job.JobStage;
+import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
+import edu.uci.ics.hyracks.comm.ConnectionManager;
+import edu.uci.ics.hyracks.comm.DemuxDataReceiveListenerFactory;
+import edu.uci.ics.hyracks.config.NCConfig;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.job.Joblet;
+import edu.uci.ics.hyracks.job.Stagelet;
+import edu.uci.ics.hyracks.runtime.OperatorRunnable;
+
+public class NodeControllerService extends AbstractRemoteService implements INodeController {
+    private static final long serialVersionUID = 1L;
+
+    private NCConfig ncConfig;
+
+    private final String id;
+
+    private final HyracksContext ctx;
+
+    private final NodeCapability nodeCapability;
+
+    private final ConnectionManager connectionManager;
+
+    private IClusterController ccs;
+
+    private Map<UUID, Joblet> jobletMap;
+
+    private Executor executor;
+
+    public NodeControllerService(NCConfig ncConfig) throws Exception {
+        this.ncConfig = ncConfig;
+        id = ncConfig.nodeId;
+        this.ctx = new HyracksContext(ncConfig.frameSize);
+        if (id == null) {
+            throw new Exception("id not set");
+        }
+        nodeCapability = computeNodeCapability();
+        connectionManager = new ConnectionManager(ctx, getIpAddress(ncConfig));
+        jobletMap = new HashMap<UUID, Joblet>();
+        executor = Executors.newCachedThreadPool();
+    }
+
+    private static Logger LOGGER = Logger.getLogger(NodeControllerService.class.getName());
+
+    @Override
+    public void start() throws Exception {
+        LOGGER.log(Level.INFO, "Starting NodeControllerService");
+        connectionManager.start();
+        Registry registry = LocateRegistry.getRegistry(ncConfig.ccHost, ncConfig.ccPort);
+        IClusterController cc = (IClusterController) registry.lookup(IClusterController.class.getName());
+        cc.registerNode(this);
+        LOGGER.log(Level.INFO, "Started NodeControllerService");
+    }
+
+    @Override
+    public void stop() throws Exception {
+        LOGGER.log(Level.INFO, "Stopping NodeControllerService");
+        connectionManager.stop();
+        LOGGER.log(Level.INFO, "Stopped NodeControllerService");
+    }
+
+    @Override
+    public String getId() throws Exception {
+        return id;
+    }
+
+    @Override
+    public NodeCapability getNodeCapability() throws Exception {
+        return nodeCapability;
+    }
+
+    public ConnectionManager getConnectionManager() {
+        return connectionManager;
+    }
+
+    private static NodeCapability computeNodeCapability() {
+        NodeCapability nc = new NodeCapability();
+        nc.setCPUCount(Runtime.getRuntime().availableProcessors());
+        return nc;
+    }
+
+    private static InetAddress getIpAddress(NCConfig ncConfig) throws Exception {
+        String ipaddrStr = ncConfig.dataIPAddress;
+        ipaddrStr = ipaddrStr.trim();
+        Pattern pattern = Pattern.compile("(\\d{1,3})\\.(\\d{1,3})\\.(\\d{1,3})\\.(\\d{1,3})");
+        Matcher m = pattern.matcher(ipaddrStr);
+        if (!m.matches()) {
+            throw new Exception(MessageFormat.format(
+                    "Connection Manager IP Address String %s does is not a valid IP Address.", ipaddrStr));
+        }
+        byte[] ipBytes = new byte[4];
+        ipBytes[0] = (byte) Integer.parseInt(m.group(1));
+        ipBytes[1] = (byte) Integer.parseInt(m.group(2));
+        ipBytes[2] = (byte) Integer.parseInt(m.group(3));
+        ipBytes[3] = (byte) Integer.parseInt(m.group(4));
+        return InetAddress.getByAddress(ipBytes);
+    }
+
+    @Override
+    public Map<PortInstanceId, Endpoint> initializeJobletPhase1(UUID jobId, JobPlan plan, JobStage stage)
+            throws Exception {
+        LOGGER.log(Level.INFO, String.valueOf(jobId) + "[" + id + ":" + stage.getId()
+                + "]: Initializing Joblet Phase 1");
+
+        final Joblet joblet = getLocalJoblet(jobId);
+
+        Stagelet stagelet = new Stagelet(joblet, stage.getId(), id);
+        joblet.setStagelet(stage.getId(), stagelet);
+
+        final Map<PortInstanceId, Endpoint> portMap = new HashMap<PortInstanceId, Endpoint>();
+        Map<OperatorInstanceId, OperatorRunnable> honMap = stagelet.getOperatorMap();
+
+        List<Endpoint> endpointList = new ArrayList<Endpoint>();
+
+        for (ActivityNodeId hanId : stage.getTasks()) {
+            IActivityNode han = plan.getActivityNodeMap().get(hanId);
+            if (LOGGER.isLoggable(Level.FINEST)) {
+                LOGGER.finest("Initializing " + hanId + " -> " + han);
+            }
+            IOperatorDescriptor op = han.getOwner();
+            List<IConnectorDescriptor> inputs = plan.getTaskInputs(hanId);
+            String[] partitions = op.getPartitions();
+            for (int i = 0; i < partitions.length; ++i) {
+                String part = partitions[i];
+                if (id.equals(part)) {
+                    IOperatorNodePushable hon = han.createPushRuntime(ctx, plan, joblet.getEnvironment(op, i), i);
+                    OperatorRunnable or = new OperatorRunnable(ctx, hon);
+                    stagelet.setOperator(op.getOperatorId(), i, or);
+                    if (inputs != null) {
+                        for (int j = 0; j < inputs.size(); ++j) {
+                            if (j >= 1) {
+                                throw new IllegalStateException();
+                            }
+                            IConnectorDescriptor conn = inputs.get(j);
+                            Endpoint endpoint = new Endpoint(connectionManager.getNetworkAddress(), i);
+                            endpointList.add(endpoint);
+                            DemuxDataReceiveListenerFactory drlf = new DemuxDataReceiveListenerFactory(ctx);
+                            connectionManager.acceptConnection(endpoint.getEndpointId(), drlf);
+                            PortInstanceId piId = new PortInstanceId(op.getOperatorId(),
+                                    PortInstanceId.Direction.INPUT, plan.getTaskInputMap().get(hanId).get(j), i);
+                            if (LOGGER.isLoggable(Level.FINEST)) {
+                                LOGGER.finest("Created endpoint " + piId + " -> " + endpoint);
+                            }
+                            portMap.put(piId, endpoint);
+                            IFrameReader reader = createReader(conn, drlf, i, plan, stagelet);
+                            or.setFrameReader(reader);
+                        }
+                    }
+                    honMap.put(new OperatorInstanceId(op.getOperatorId(), i), or);
+                }
+            }
+        }
+
+        stagelet.setEndpointList(endpointList);
+
+        return portMap;
+    }
+
+    private IFrameReader createReader(final IConnectorDescriptor conn, IConnectionDemultiplexer demux,
+            final int receiverIndex, JobPlan plan, final Stagelet stagelet) throws HyracksDataException {
+        final IFrameReader reader = conn.createReceiveSideReader(ctx, plan, demux, receiverIndex);
+
+        return plan.getJobFlags().contains(JobFlag.COLLECT_FRAME_COUNTS) ? new IFrameReader() {
+            private int frameCount;
+
+            @Override
+            public void open() throws HyracksDataException {
+                frameCount = 0;
+                reader.open();
+            }
+
+            @Override
+            public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                boolean status = reader.nextFrame(buffer);
+                if (status) {
+                    ++frameCount;
+                }
+                return status;
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                reader.close();
+                stagelet.getStatistics().getStatisticsMap().put(
+                        "framecount." + conn.getConnectorId().getId() + ".receiver." + receiverIndex,
+                        String.valueOf(frameCount));
+            }
+        } : reader;
+    }
+
+    @Override
+    public void initializeJobletPhase2(UUID jobId, final JobPlan plan, JobStage stage,
+            final Map<PortInstanceId, Endpoint> globalPortMap) throws Exception {
+        LOGGER.log(Level.INFO, String.valueOf(jobId) + "[" + id + ":" + stage.getId()
+                + "]: Initializing Joblet Phase 2");
+        final Joblet ji = getLocalJoblet(jobId);
+        Stagelet si = (Stagelet) ji.getStagelet(stage.getId());
+        final Map<OperatorInstanceId, OperatorRunnable> honMap = si.getOperatorMap();
+
+        final Stagelet stagelet = (Stagelet) ji.getStagelet(stage.getId());
+
+        final JobSpecification spec = plan.getJobSpecification();
+
+        for (ActivityNodeId hanId : stage.getTasks()) {
+            IActivityNode han = plan.getActivityNodeMap().get(hanId);
+            IOperatorDescriptor op = han.getOwner();
+            List<IConnectorDescriptor> outputs = plan.getTaskOutputs(hanId);
+            String[] partitions = op.getPartitions();
+            for (int i = 0; i < partitions.length; ++i) {
+                String part = partitions[i];
+                if (id.equals(part)) {
+                    OperatorRunnable or = honMap.get(new OperatorInstanceId(op.getOperatorId(), i));
+                    if (outputs != null) {
+                        for (int j = 0; j < outputs.size(); ++j) {
+                            final IConnectorDescriptor conn = outputs.get(j);
+                            final int senderIndex = i;
+                            IEndpointDataWriterFactory edwFactory = new IEndpointDataWriterFactory() {
+                                @Override
+                                public IFrameWriter createFrameWriter(int index) throws HyracksDataException {
+                                    PortInstanceId piId = new PortInstanceId(spec.getConsumer(conn).getOperatorId(),
+                                            PortInstanceId.Direction.INPUT, spec.getConsumerInputIndex(conn), index);
+                                    Endpoint ep = globalPortMap.get(piId);
+                                    if (LOGGER.isLoggable(Level.FINEST)) {
+                                        LOGGER.finest("Probed endpoint " + piId + " -> " + ep);
+                                    }
+                                    return createWriter(connectionManager.connect(ep.getNetworkAddress(), ep
+                                            .getEndpointId(), senderIndex), plan, conn, senderIndex, index, stagelet);
+                                }
+                            };
+                            or.setFrameWriter(j, conn.createSendSideWriter(ctx, plan, edwFactory, i));
+                        }
+                    }
+                    stagelet.installRunnable(new OperatorInstanceId(op.getOperatorId(), i));
+                }
+            }
+        }
+    }
+
+    private IFrameWriter createWriter(final IFrameWriter writer, JobPlan plan, final IConnectorDescriptor conn,
+            final int senderIndex, final int receiverIndex, final Stagelet stagelet) throws HyracksDataException {
+        return plan.getJobFlags().contains(JobFlag.COLLECT_FRAME_COUNTS) ? new IFrameWriter() {
+            private int frameCount;
+
+            @Override
+            public void open() throws HyracksDataException {
+                frameCount = 0;
+                writer.open();
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                ++frameCount;
+                writer.nextFrame(buffer);
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                writer.close();
+                stagelet.getStatistics().getStatisticsMap().put(
+                        "framecount." + conn.getConnectorId().getId() + ".sender." + senderIndex + "." + receiverIndex,
+                        String.valueOf(frameCount));
+            }
+        } : writer;
+    }
+
+    @Override
+    public void commitJobletInitialization(UUID jobId, JobPlan plan, JobStage stage) throws Exception {
+        final Joblet ji = getLocalJoblet(jobId);
+        Stagelet si = (Stagelet) ji.getStagelet(stage.getId());
+        for (Endpoint e : si.getEndpointList()) {
+            connectionManager.unacceptConnection(e.getEndpointId());
+        }
+        si.setEndpointList(null);
+    }
+
+    private synchronized Joblet getLocalJoblet(UUID jobId) throws Exception {
+        Joblet ji = jobletMap.get(jobId);
+        if (ji == null) {
+            ji = new Joblet(this, jobId);
+            jobletMap.put(jobId, ji);
+        }
+        return ji;
+    }
+
+    public Executor getExecutor() {
+        return executor;
+    }
+
+    @Override
+    public synchronized void cleanUpJob(UUID jobId) throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Cleaning up after job: " + jobId);
+        }
+        jobletMap.remove(jobId);
+        connectionManager.dumpStats();
+    }
+
+    @Override
+    public void startStage(UUID jobId, UUID stageId) throws Exception {
+        Joblet ji = jobletMap.get(jobId);
+        if (ji != null) {
+            Stagelet s = ji.getStagelet(stageId);
+            if (s != null) {
+                s.start();
+            }
+        }
+    }
+
+    public void notifyStageComplete(UUID jobId, UUID stageId, StageletStatistics stats) throws Exception {
+        ccs.notifyStageletComplete(jobId, stageId, id, stats);
+    }
+
+    @Override
+    public void notifyRegistration(IClusterController ccs) throws Exception {
+        this.ccs = ccs;
+    }
+
+	@Override
+	public NCConfig getConfiguration() throws Exception {
+		return ncConfig;
+	}
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/ExternalSortOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/ExternalSortOperatorDescriptor.java
new file mode 100644
index 0000000..e5052b2
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/ExternalSortOperatorDescriptor.java
@@ -0,0 +1,643 @@
+/*
+ * 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.coreops;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.coreops.base.AbstractOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.util.ReferenceEntry;
+import edu.uci.ics.hyracks.coreops.util.ReferencedPriorityQueue;
+
+public class ExternalSortOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final String IN_FRAMES = "inFrames";
+    private static final String TPOINTERS = "tPointers";
+    private static final String RUNS = "runs";
+
+    private static final long serialVersionUID = 1L;
+    private final int[] sortFields;
+    private IBinaryComparatorFactory[] comparatorFactories;
+    private final int framesLimit;
+
+    public ExternalSortOperatorDescriptor(JobSpecification spec, int framesLimit, int[] sortFields,
+            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.framesLimit = framesLimit;
+        this.sortFields = sortFields;
+        this.comparatorFactories = comparatorFactories;
+        if (framesLimit <= 1) {
+            throw new IllegalStateException();// minimum of 2 fames (1 in,1 out)
+        }
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        SortActivity sa = new SortActivity();
+        MergeActivity ma = new MergeActivity();
+
+        builder.addTask(sa);
+        builder.addSourceEdge(0, sa, 0);
+
+        builder.addTask(ma);
+        builder.addTargetEdge(0, ma, 0);
+
+        builder.addBlockingEdge(sa, ma);
+    }
+
+    private class SortActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return ExternalSortOperatorDescriptor.this;
+        }
+
+        @Override
+        public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+                int partition) {
+            return null;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final HyracksContext ctx, JobPlan plan,
+                final IOperatorEnvironment env, int partition) {
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private final FrameTupleAccessor fta1 = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                private final FrameTupleAccessor fta2 = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                private List<ByteBuffer> inFrames;
+                private ByteBuffer outFrame;
+                private LinkedList<File> runs;
+                private int activeInFrame;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    inFrames = new ArrayList<ByteBuffer>();
+                    outFrame = ctx.getResourceManager().allocateFrame();
+                    runs = new LinkedList<File>();
+                    activeInFrame = 0;
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    if (activeInFrame + 1 >= framesLimit) { // + 1 outFrame.
+                        try {
+                            createRunFromInFrames(inFrames.size());
+                        } catch (Exception e) {
+                            e.printStackTrace();
+                        }
+                    }
+                    ByteBuffer copy;
+                    buffer.position(0);
+                    buffer.limit(buffer.capacity());
+                    if (runs.size() <= 0) {
+                        copy = ctx.getResourceManager().allocateFrame();
+                        copy.put(buffer);
+                        inFrames.add(copy);
+                    } else {
+                        copy = inFrames.get(activeInFrame);
+                        copy.put(buffer);
+                    }
+                    ++activeInFrame;
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    env.set(IN_FRAMES, inFrames);
+                    env.set(RUNS, runs);
+                    if (activeInFrame > 0) {
+                        if (runs.size() <= 0) {
+                            long[] tPointers = getSortedTPointers(activeInFrame);
+                            env.set(TPOINTERS, tPointers);
+                        } else {
+                            createRunFromInFrames(activeInFrame);
+                        }
+                    }
+                }
+
+                private void createRunFromInFrames(int nBuffers) throws HyracksDataException {
+                    File runFile;
+                    try {
+                        runFile = ctx.getResourceManager().createFile(
+                                ExternalSortOperatorDescriptor.class.getSimpleName(), ".run");
+                    } catch (IOException e) {
+                        throw new HyracksDataException(e);
+                    }
+                    RunFileWriter writer = new RunFileWriter(runFile);
+                    writer.open();
+                    try {
+                        flushFrames(ctx, inFrames, outFrame, getSortedTPointers(nBuffers), writer);
+                    } finally {
+                        writer.close();
+                    }
+                    runs.add(runFile);
+                    activeInFrame = 0;
+                }
+
+                private long[] getSortedTPointers(int nBuffers) {
+                    FrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                    int totalTCount = 0;
+                    for (int i = 0; i < nBuffers; ++i) {
+                        accessor.reset(inFrames.get(i));
+                        totalTCount += accessor.getTupleCount();
+                    }
+                    long[] tPointers = new long[totalTCount];
+                    int ptr = 0;
+                    for (int i = 0; i < nBuffers; ++i) {
+                        accessor.reset(inFrames.get(i));
+                        int tCount = accessor.getTupleCount();
+                        for (int j = 0; j < tCount; ++j) {
+                            tPointers[ptr++] = (((long) i) << 32) + j;
+                        }
+                    }
+                    if (tPointers.length > 0) {
+                        sort(tPointers, 0, tPointers.length);
+                    }
+                    return tPointers;
+                }
+
+                private void sort(long[] tPointers, int offset, int length) {
+                    int m = offset + (length >> 1);
+                    long v = tPointers[m];
+
+                    int a = offset;
+                    int b = a;
+                    int c = offset + length - 1;
+                    int d = c;
+                    while (true) {
+                        while (b <= c && compare(tPointers[b], v) <= 0) {
+                            if (compare(tPointers[b], v) == 0) {
+                                swap(tPointers, a++, b);
+                            }
+                            ++b;
+                        }
+                        while (c >= b && compare(tPointers[c], v) >= 0) {
+                            if (compare(tPointers[c], v) == 0) {
+                                swap(tPointers, c, d--);
+                            }
+                            --c;
+                        }
+                        if (b > c)
+                            break;
+                        swap(tPointers, b++, c--);
+                    }
+
+                    int s;
+                    int n = offset + length;
+                    s = Math.min(a - offset, b - a);
+                    vecswap(tPointers, offset, b - s, s);
+                    s = Math.min(d - c, n - d - 1);
+                    vecswap(tPointers, b, n - s, s);
+
+                    if ((s = b - a) > 1) {
+                        sort(tPointers, offset, s);
+                    }
+                    if ((s = d - c) > 1) {
+                        sort(tPointers, n - s, s);
+                    }
+                }
+
+                private void swap(long x[], int a, int b) {
+                    long t = x[a];
+                    x[a] = x[b];
+                    x[b] = t;
+                }
+
+                private void vecswap(long x[], int a, int b, int n) {
+                    for (int i = 0; i < n; i++, a++, b++) {
+                        swap(x, a, b);
+                    }
+                }
+
+                private int compare(long tp1, long tp2) {
+                    int i1 = (int) ((tp1 >> 32) & 0xffffffff);
+                    int j1 = (int) (tp1 & 0xffffffff);
+                    int i2 = (int) ((tp2 >> 32) & 0xffffffff);
+                    int j2 = (int) (tp2 & 0xffffffff);
+                    ByteBuffer buf1 = inFrames.get(i1);
+                    ByteBuffer buf2 = inFrames.get(i2);
+                    byte[] b1 = buf1.array();
+                    byte[] b2 = buf2.array();
+                    fta1.reset(buf1);
+                    fta2.reset(buf2);
+                    for (int f = 0; f < sortFields.length; ++f) {
+                        int fIdx = sortFields[f];
+                        int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
+                                + fta1.getFieldStartOffset(j1, fIdx);
+                        int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
+                        int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
+                                + fta2.getFieldStartOffset(j2, fIdx);
+                        int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
+                        int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+                        if (c != 0) {
+                            return c;
+                        }
+                    }
+                    return 0;
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer) {
+                    throw new IllegalArgumentException();
+                }
+            };
+            return op;
+        }
+
+        @Override
+        public boolean supportsPullInterface() {
+            return false;
+        }
+
+        @Override
+        public boolean supportsPushInterface() {
+            return false;
+        }
+    }
+
+    private class MergeActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return ExternalSortOperatorDescriptor.this;
+        }
+
+        @Override
+        public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+                int partition) {
+            return null;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final HyracksContext ctx, JobPlan plan,
+                final IOperatorEnvironment env, int partition) {
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private IFrameWriter writer;
+                private List<ByteBuffer> inFrames;
+                private ByteBuffer outFrame;
+                LinkedList<File> runs;
+                private FrameTupleAppender outFrameAppender;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    inFrames = (List<ByteBuffer>) env.get(IN_FRAMES);
+                    outFrame = ctx.getResourceManager().allocateFrame();
+                    runs = (LinkedList<File>) env.get(RUNS);
+                    outFrameAppender = new FrameTupleAppender(ctx);
+                    outFrameAppender.reset(outFrame, true);
+                    writer.open();
+                    try {
+                        if (runs.size() <= 0) {
+                            long[] tPointers = (long[]) env.get(TPOINTERS);
+                            if (tPointers != null) {
+                                flushFrames(ctx, inFrames, outFrame, tPointers, writer);
+                                env.set(TPOINTERS, null);
+                            }
+                        } else {
+                            int passCount = 0;
+                            while (runs.size() > 0) {
+                                passCount++;
+                                try {
+                                    doPass(runs, passCount);
+                                } catch (Exception e) {
+                                    e.printStackTrace();
+                                }
+                            }
+                        }
+                    } finally {
+                        writer.close();
+                    }
+                    env.set(IN_FRAMES, null);
+                    env.set(RUNS, null);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    throw new IllegalStateException();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    // do nothing
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer) {
+                    if (index != 0) {
+                        throw new IllegalArgumentException();
+                    }
+                    this.writer = writer;
+                }
+
+                // creates a new run from runs that can fit in memory.
+                private void doPass(LinkedList<File> runs, int passCount) throws ClassNotFoundException, Exception {
+                    File newRun = null;
+                    IFrameWriter writer = this.writer;
+                    boolean finalPass = false;
+                    if (runs.size() + 1 <= framesLimit) { // + 1 outFrame
+                        finalPass = true;
+                        for (int i = inFrames.size() - 1; i >= runs.size(); i--) {
+                            inFrames.remove(i);
+                        }
+                    } else {
+                        newRun = ctx.getResourceManager().createFile(
+                                ExternalSortOperatorDescriptor.class.getSimpleName(), ".run");
+                        writer = new RunFileWriter(newRun);
+                    }
+                    RunFileReader[] runCursors = new RunFileReader[inFrames.size()];
+                    FrameTupleAccessor[] tupleAccessors = new FrameTupleAccessor[inFrames.size()];
+                    Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
+                    ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(ctx, recordDescriptors[0],
+                            inFrames.size(), comparator);
+                    int[] tupleIndexes = new int[inFrames.size()];
+                    for (int i = 0; i < inFrames.size(); i++) {
+                        tupleIndexes[i] = 0;
+                        int runIndex = topTuples.peek().getRunid();
+                        runCursors[runIndex] = new RunFileReader(runs.get(runIndex));
+                        runCursors[runIndex].open();
+                        if (runCursors[runIndex].nextFrame(inFrames.get(runIndex))) {
+                            tupleAccessors[runIndex] = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                            tupleAccessors[runIndex].reset(inFrames.get(runIndex));
+                            setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+                        } else {
+                            closeRun(runIndex, runCursors, tupleAccessors);
+                        }
+                    }
+
+                    while (!topTuples.areRunsExhausted()) {
+                        ReferenceEntry top = topTuples.peek();
+                        int runIndex = top.getRunid();
+                        FrameTupleAccessor fta = top.getAccessor();
+                        int tupleIndex = top.getTupleIndex();
+
+                        if (!outFrameAppender.append(fta, tupleIndex)) {
+                            flushFrame(outFrame, writer);
+                            outFrameAppender.reset(outFrame, true);
+                            if (!outFrameAppender.append(fta, tupleIndex)) {
+                                throw new IllegalStateException();
+                            }
+                        }
+
+                        ++tupleIndexes[runIndex];
+                        setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+                    }
+                    if (outFrameAppender.getTupleCount() > 0) {
+                        flushFrame(outFrame, writer);
+                        outFrameAppender.reset(outFrame, true);
+                    }
+                    runs.subList(0, inFrames.size()).clear();
+                    if (!finalPass) {
+                        runs.add(0, newRun);
+                    }
+                }
+
+                private void setNextTopTuple(int runIndex, int[] tupleIndexes, RunFileReader[] runCursors,
+                        FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws IOException {
+                    boolean exists = hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+                    if (exists) {
+                        topTuples.popAndReplace(tupleAccessors[runIndex], tupleIndexes[runIndex]);
+                    } else {
+                        topTuples.pop();
+                        closeRun(runIndex, runCursors, tupleAccessors);
+                    }
+                }
+
+                private boolean hasNextTuple(int runIndex, int[] tupleIndexes, RunFileReader[] runCursors,
+                        FrameTupleAccessor[] tupleAccessors) throws IOException {
+                    if (tupleAccessors[runIndex] == null || runCursors[runIndex] == null) {
+                        return false;
+                    } else if (tupleIndexes[runIndex] >= tupleAccessors[runIndex].getTupleCount()) {
+                        ByteBuffer buf = tupleAccessors[runIndex].getBuffer(); // same-as-inFrames.get(runIndex)
+                        if (runCursors[runIndex].nextFrame(buf)) {
+                            tupleIndexes[runIndex] = 0;
+                            return hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+                        } else {
+                            return false;
+                        }
+                    } else {
+                        return true;
+                    }
+                }
+
+                private void closeRun(int index, RunFileReader[] runCursors, FrameTupleAccessor[] tupleAccessor) {
+                    runCursors[index] = null;
+                    tupleAccessor[index] = null;
+                }
+            };
+            return op;
+        }
+
+        @Override
+        public boolean supportsPullInterface() {
+            return false;
+        }
+
+        @Override
+        public boolean supportsPushInterface() {
+            return true;
+        }
+    }
+
+    private Comparator<ReferenceEntry> createEntryComparator(final IBinaryComparator[] comparators) {
+        return new Comparator<ReferenceEntry>() {
+            public int compare(ReferenceEntry tp1, ReferenceEntry tp2) {
+                FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
+                FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
+                int j1 = (Integer) tp1.getTupleIndex();
+                int j2 = (Integer) tp2.getTupleIndex();
+                byte[] b1 = fta1.getBuffer().array();
+                byte[] b2 = fta2.getBuffer().array();
+                for (int f = 0; f < sortFields.length; ++f) {
+                    int fIdx = sortFields[f];
+                    int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
+                            + fta1.getFieldStartOffset(j1, fIdx);
+                    int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
+                    int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
+                            + fta2.getFieldStartOffset(j2, fIdx);
+                    int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
+                    int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+                    if (c != 0) {
+                        return c;
+                    }
+                }
+                return 0;
+            }
+        };
+    }
+
+    private void flushFrames(HyracksContext ctx, List<ByteBuffer> inFrames, ByteBuffer outFrame, long[] tPointers,
+            IFrameWriter writer) throws HyracksDataException {
+        FrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+        FrameTupleAppender outFrameAppender = new FrameTupleAppender(ctx);
+        for (ByteBuffer buf : inFrames) {
+            buf.position(0);
+            buf.limit(buf.capacity());
+        }
+        outFrameAppender.reset(outFrame, true);
+        for (int ptr = 0; ptr < tPointers.length; ++ptr) {
+            long tp = tPointers[ptr];
+            int i = (int) ((tp >> 32) & 0xffffffff);
+            int j = (int) (tp & 0xffffffff);
+            ByteBuffer buffer = inFrames.get(i);
+            accessor.reset(buffer);
+            if (!outFrameAppender.append(accessor, j)) {
+                flushFrame(outFrame, writer);
+                outFrameAppender.reset(outFrame, true);
+                if (!outFrameAppender.append(accessor, j)) {
+                    throw new IllegalStateException();
+                }
+            }
+        }
+        if (outFrameAppender.getTupleCount() > 0) {
+            flushFrame(outFrame, writer);
+            outFrame.position(0);
+            outFrame.limit(outFrame.capacity());
+        }
+    }
+
+    private void flushFrame(ByteBuffer frame, IFrameWriter writer) throws HyracksDataException {
+        frame.position(0);
+        frame.limit(frame.capacity());
+        writer.nextFrame(frame);
+    }
+
+    private class RunFileWriter implements IFrameWriter {
+        private final File file;
+        private FileChannel channel;
+
+        public RunFileWriter(File file) {
+            this.file = file;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            RandomAccessFile raf;
+            try {
+                raf = new RandomAccessFile(file, "rw");
+            } catch (FileNotFoundException e) {
+                throw new HyracksDataException(e);
+            }
+            channel = raf.getChannel();
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            int remain = buffer.capacity();
+            while (remain > 0) {
+                int len;
+                try {
+                    len = channel.write(buffer);
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+                if (len < 0) {
+                    throw new HyracksDataException("Error writing data");
+                }
+                remain -= len;
+            }
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            try {
+                channel.close();
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    public static class RunFileReader implements IFrameReader {
+        private final File file;
+        private FileChannel channel;
+
+        public RunFileReader(File file) throws FileNotFoundException {
+            this.file = file;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            RandomAccessFile raf;
+            try {
+                raf = new RandomAccessFile(file, "r");
+            } catch (FileNotFoundException e) {
+                throw new HyracksDataException(e);
+            }
+            channel = raf.getChannel();
+        }
+
+        @Override
+        public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            buffer.clear();
+            int remain = buffer.capacity();
+            while (remain > 0) {
+                int len;
+                try {
+                    len = channel.read(buffer);
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+                if (len < 0) {
+                    return false;
+                }
+                remain -= len;
+            }
+            return true;
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            try {
+                channel.close();
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/FieldHashPartitionComputerFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/FieldHashPartitionComputerFactory.java
new file mode 100644
index 0000000..ab66fb4
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/FieldHashPartitionComputerFactory.java
@@ -0,0 +1,61 @@
+/*
+ * 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.coreops;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+
+public class FieldHashPartitionComputerFactory implements ITuplePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+    private final int[] hashFields;
+    private final IBinaryHashFunctionFactory[] hashFunctionFactories;
+
+    public FieldHashPartitionComputerFactory(int[] hashFields, IBinaryHashFunctionFactory[] hashFunctionFactories) {
+        this.hashFields = hashFields;
+        this.hashFunctionFactories = hashFunctionFactories;
+    }
+
+    @Override
+    public ITuplePartitionComputer createPartitioner() {
+        final IBinaryHashFunction[] hashFunctions = new IBinaryHashFunction[hashFunctionFactories.length];
+        for (int i = 0; i < hashFunctionFactories.length; ++i) {
+            hashFunctions[i] = hashFunctionFactories[i].createBinaryHashFunction();
+        }
+        return new ITuplePartitionComputer() {
+            @Override
+            public int partition(FrameTupleAccessor accessor, int tIndex, int nParts) {
+                int h = 0;
+                int startOffset = accessor.getTupleStartOffset(tIndex);
+                int slotLength = accessor.getFieldSlotsLength();
+                for (int j = 0; j < hashFields.length; ++j) {
+                    int fIdx = hashFields[j];
+                    IBinaryHashFunction hashFn = hashFunctions[j];
+                    int fStart = accessor.getFieldStartOffset(tIndex, fIdx);
+                    int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
+                    int fh = hashFn
+                            .hash(accessor.getBuffer().array(), startOffset + slotLength + fStart, fEnd - fStart);
+                    h += fh;
+                }
+                if (h < 0) {
+                    h = -h;
+                }
+                return h % nParts;
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/HashDataWriter.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/HashDataWriter.java
new file mode 100644
index 0000000..2c1e28a
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/HashDataWriter.java
@@ -0,0 +1,95 @@
+/*
+ * 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.coreops;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class HashDataWriter implements IFrameWriter {
+    private final int consumerPartitionCount;
+    private final IFrameWriter[] epWriters;
+    private final FrameTupleAppender[] appenders;
+    private final FrameTupleAccessor tupleAccessor;
+    private final ITuplePartitionComputer tpc;
+
+    public HashDataWriter(HyracksContext ctx, int consumerPartitionCount, IEndpointDataWriterFactory edwFactory,
+            RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc) throws HyracksDataException {
+        this.consumerPartitionCount = consumerPartitionCount;
+        epWriters = new IFrameWriter[consumerPartitionCount];
+        appenders = new FrameTupleAppender[consumerPartitionCount];
+        for (int i = 0; i < consumerPartitionCount; ++i) {
+            try {
+                epWriters[i] = edwFactory.createFrameWriter(i);
+                appenders[i] = new FrameTupleAppender(ctx);
+                appenders[i].reset(ctx.getResourceManager().allocateFrame(), true);
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        tupleAccessor = new FrameTupleAccessor(ctx, recordDescriptor);
+        this.tpc = tpc;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        for (int i = 0; i < epWriters.length; ++i) {
+            if (appenders[i].getTupleCount() > 0) {
+                flushFrame(appenders[i].getBuffer(), epWriters[i]);
+            }
+            epWriters[i].close();
+        }
+    }
+
+    private void flushFrame(ByteBuffer buffer, IFrameWriter frameWriter) throws HyracksDataException {
+        buffer.position(0);
+        buffer.limit(buffer.capacity());
+        frameWriter.nextFrame(buffer);
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        for (int i = 0; i < epWriters.length; ++i) {
+            epWriters[i].open();
+            appenders[i].reset(appenders[i].getBuffer(), true);
+        }
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        tupleAccessor.reset(buffer);
+        int tupleCount = tupleAccessor.getTupleCount();
+        for (int i = 0; i < tupleCount; ++i) {
+            int h = tpc.partition(tupleAccessor, i, consumerPartitionCount);
+            FrameTupleAppender appender = appenders[h];
+            if (!appender.append(tupleAccessor, i)) {
+                ByteBuffer appenderBuffer = appender.getBuffer();
+                flushFrame(appenderBuffer, epWriters[h]);
+                appender.reset(appenderBuffer, true);
+                if (!appender.append(tupleAccessor, i)) {
+                    throw new IllegalStateException();
+                }
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/IMapper.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/IMapper.java
new file mode 100644
index 0000000..aec6135
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/IMapper.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.coreops;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IMapper {
+    public void map(Object[] data, IDataWriter<Object[]> writer) throws HyracksDataException;
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/IMapperFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/IMapperFactory.java
new file mode 100644
index 0000000..d73a3fc
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/IMapperFactory.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IMapperFactory extends Serializable {
+    public IMapper createMapper() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/ITupleAggregator.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/ITupleAggregator.java
new file mode 100644
index 0000000..c482591
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/ITupleAggregator.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.coreops;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ITupleAggregator {
+    void add(Object[] data);
+
+    void init(Object[] data);
+
+    void write(IDataWriter<Object[]> writer) throws HyracksDataException;
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/ITupleAggregatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/ITupleAggregatorFactory.java
new file mode 100644
index 0000000..161232b
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/ITupleAggregatorFactory.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ITupleAggregatorFactory extends Serializable {
+    public ITupleAggregator createTupleAggregator() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/InMemorySortOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/InMemorySortOperatorDescriptor.java
new file mode 100644
index 0000000..45e882a
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/InMemorySortOperatorDescriptor.java
@@ -0,0 +1,321 @@
+/*
+ * 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.coreops;
+
+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.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.coreops.base.AbstractOperatorDescriptor;
+
+public class InMemorySortOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final String BUFFERS = "buffers";
+    private static final String TPOINTERS = "tpointers";
+
+    private static final long serialVersionUID = 1L;
+    private final int[] sortFields;
+    private IBinaryComparatorFactory[] comparatorFactories;
+
+    public InMemorySortOperatorDescriptor(JobSpecification spec, int[] sortFields,
+            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.sortFields = sortFields;
+        this.comparatorFactories = comparatorFactories;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        SortActivity sa = new SortActivity();
+        MergeActivity ma = new MergeActivity();
+
+        builder.addTask(sa);
+        builder.addSourceEdge(0, sa, 0);
+
+        builder.addTask(ma);
+        builder.addTargetEdge(0, ma, 0);
+
+        builder.addBlockingEdge(sa, ma);
+    }
+
+    private class SortActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return InMemorySortOperatorDescriptor.this;
+        }
+
+        @Override
+        public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+                int partition) {
+            return null;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final HyracksContext ctx, JobPlan plan,
+                final IOperatorEnvironment env, int partition) {
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private List<ByteBuffer> buffers;
+
+                private final FrameTupleAccessor fta1 = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                private final FrameTupleAccessor fta2 = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+
+                @Override
+                public void open() throws HyracksDataException {
+                    buffers = new ArrayList<ByteBuffer>();
+                    env.set(BUFFERS, buffers);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    ByteBuffer copy = ctx.getResourceManager().allocateFrame();
+                    FrameUtils.copy(buffer, copy);
+                    buffers.add(copy);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    FrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                    int nBuffers = buffers.size();
+                    int totalTCount = 0;
+                    for (int i = 0; i < nBuffers; ++i) {
+                        accessor.reset(buffers.get(i));
+                        totalTCount += accessor.getTupleCount();
+                    }
+                    long[] tPointers = new long[totalTCount];
+                    int ptr = 0;
+                    for (int i = 0; i < nBuffers; ++i) {
+                        accessor.reset(buffers.get(i));
+                        int tCount = accessor.getTupleCount();
+                        for (int j = 0; j < tCount; ++j) {
+                            tPointers[ptr++] = (((long) i) << 32) + j;
+                        }
+                    }
+                    if (tPointers.length > 0) {
+                        sort(tPointers, 0, tPointers.length);
+                    }
+                    env.set(TPOINTERS, tPointers);
+                }
+
+                private void sort(long[] tPointers, int offset, int length) {
+                    int m = offset + (length >> 1);
+                    long v = tPointers[m];
+
+                    int a = offset;
+                    int b = a;
+                    int c = offset + length - 1;
+                    int d = c;
+                    while (true) {
+                        while (b <= c && compare(tPointers[b], v) <= 0) {
+                            if (compare(tPointers[b], v) == 0) {
+                                swap(tPointers, a++, b);
+                            }
+                            ++b;
+                        }
+                        while (c >= b && compare(tPointers[c], v) >= 0) {
+                            if (compare(tPointers[c], v) == 0) {
+                                swap(tPointers, c, d--);
+                            }
+                            --c;
+                        }
+                        if (b > c)
+                            break;
+                        swap(tPointers, b++, c--);
+                    }
+
+                    int s;
+                    int n = offset + length;
+                    s = Math.min(a - offset, b - a);
+                    vecswap(tPointers, offset, b - s, s);
+                    s = Math.min(d - c, n - d - 1);
+                    vecswap(tPointers, b, n - s, s);
+
+                    if ((s = b - a) > 1) {
+                        sort(tPointers, offset, s);
+                    }
+                    if ((s = d - c) > 1) {
+                        sort(tPointers, n - s, s);
+                    }
+                }
+
+                private void swap(long x[], int a, int b) {
+                    long t = x[a];
+                    x[a] = x[b];
+                    x[b] = t;
+                }
+
+                private void vecswap(long x[], int a, int b, int n) {
+                    for (int i = 0; i < n; i++, a++, b++) {
+                        swap(x, a, b);
+                    }
+                }
+
+                private int compare(long tp1, long tp2) {
+                    int i1 = (int) ((tp1 >> 32) & 0xffffffff);
+                    int j1 = (int) (tp1 & 0xffffffff);
+                    int i2 = (int) ((tp2 >> 32) & 0xffffffff);
+                    int j2 = (int) (tp2 & 0xffffffff);
+                    ByteBuffer buf1 = buffers.get(i1);
+                    ByteBuffer buf2 = buffers.get(i2);
+                    byte[] b1 = buf1.array();
+                    byte[] b2 = buf2.array();
+                    fta1.reset(buf1);
+                    fta2.reset(buf2);
+                    for (int f = 0; f < sortFields.length; ++f) {
+                        int fIdx = sortFields[f];
+                        int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
+                                + fta1.getFieldStartOffset(j1, fIdx);
+                        int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
+                        int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
+                                + fta2.getFieldStartOffset(j2, fIdx);
+                        int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
+                        int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+                        if (c != 0) {
+                            return c;
+                        }
+                    }
+                    return 0;
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer) {
+                    throw new IllegalArgumentException();
+                }
+            };
+            return op;
+        }
+
+        @Override
+        public boolean supportsPullInterface() {
+            return false;
+        }
+
+        @Override
+        public boolean supportsPushInterface() {
+            return false;
+        }
+    }
+
+    private class MergeActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return InMemorySortOperatorDescriptor.this;
+        }
+
+        @Override
+        public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+                int partition) {
+            return null;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final HyracksContext ctx, JobPlan plan,
+                final IOperatorEnvironment env, int partition) {
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private IFrameWriter writer;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    List<ByteBuffer> buffers = (List<ByteBuffer>) env.get(BUFFERS);
+                    long[] tPointers = (long[]) env.get(TPOINTERS);
+                    FrameTupleAccessor accessor = new FrameTupleAccessor(ctx, recordDescriptors[0]);
+                    FrameTupleAppender appender = new FrameTupleAppender(ctx);
+                    ByteBuffer outFrame = ctx.getResourceManager().allocateFrame();
+                    writer.open();
+                    appender.reset(outFrame, true);
+                    for (int ptr = 0; ptr < tPointers.length; ++ptr) {
+                        long tp = tPointers[ptr];
+                        int i = (int) ((tp >> 32) & 0xffffffff);
+                        int j = (int) (tp & 0xffffffff);
+                        ByteBuffer buffer = buffers.get(i);
+                        accessor.reset(buffer);
+                        if (!appender.append(accessor, j)) {
+                            flushFrame(outFrame);
+                            appender.reset(outFrame, true);
+                            if (!appender.append(accessor, j)) {
+                                throw new IllegalStateException();
+                            }
+                        }
+                    }
+                    if (appender.getTupleCount() > 0) {
+                        flushFrame(outFrame);
+                    }
+                    writer.close();
+                    env.set(BUFFERS, null);
+                    env.set(TPOINTERS, null);
+                }
+
+                private void flushFrame(ByteBuffer frame) throws HyracksDataException {
+                    frame.position(0);
+                    frame.limit(frame.capacity());
+                    writer.nextFrame(frame);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    throw new IllegalStateException();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    // do nothing
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer) {
+                    if (index != 0) {
+                        throw new IllegalArgumentException();
+                    }
+                    this.writer = writer;
+                }
+            };
+            return op;
+        }
+
+        @Override
+        public boolean supportsPullInterface() {
+            return false;
+        }
+
+        @Override
+        public boolean supportsPushInterface() {
+            return true;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MToNHashPartitioningConnectorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MToNHashPartitioningConnectorDescriptor.java
new file mode 100644
index 0000000..937e0a4
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MToNHashPartitioningConnectorDescriptor.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.coreops;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.NonDeterministicFrameReader;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractConnectorDescriptor;
+
+public class MToNHashPartitioningConnectorDescriptor extends AbstractConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private ITuplePartitionComputerFactory tpcf;
+
+    public MToNHashPartitioningConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf) {
+        super(spec);
+        this.tpcf = tpcf;
+    }
+
+    @Override
+    public IFrameWriter createSendSideWriter(HyracksContext ctx, JobPlan plan, IEndpointDataWriterFactory edwFactory,
+            int index) throws HyracksDataException {
+        JobSpecification spec = plan.getJobSpecification();
+        final int consumerPartitionCount = spec.getConsumer(this).getPartitions().length;
+        final HashDataWriter hashWriter = new HashDataWriter(ctx, consumerPartitionCount, edwFactory, spec
+                .getConnectorRecordDescriptor(this), tpcf.createPartitioner());
+        return hashWriter;
+    }
+
+    @Override
+    public IFrameReader createReceiveSideReader(HyracksContext ctx, JobPlan plan, IConnectionDemultiplexer demux,
+            int index) throws HyracksDataException {
+        return new NonDeterministicFrameReader(ctx, demux);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MToNHashPartitioningMergingConnectorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MToNHashPartitioningMergingConnectorDescriptor.java
new file mode 100644
index 0000000..9f36730
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MToNHashPartitioningMergingConnectorDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.SortMergeFrameReader;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractConnectorDescriptor;
+
+public class MToNHashPartitioningMergingConnectorDescriptor extends AbstractConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final ITuplePartitionComputerFactory tpcf;
+    private final int[] sortFields;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+
+    public MToNHashPartitioningMergingConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf,
+            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories) {
+        super(spec);
+        this.tpcf = tpcf;
+        this.sortFields = sortFields;
+        this.comparatorFactories = comparatorFactories;
+    }
+
+    @Override
+    public IFrameWriter createSendSideWriter(HyracksContext ctx, JobPlan plan, IEndpointDataWriterFactory edwFactory,
+            int index) throws HyracksDataException {
+        JobSpecification spec = plan.getJobSpecification();
+        final int consumerPartitionCount = spec.getConsumer(this).getPartitions().length;
+        final HashDataWriter hashWriter = new HashDataWriter(ctx, consumerPartitionCount, edwFactory, spec
+                .getConnectorRecordDescriptor(this), tpcf.createPartitioner());
+        return hashWriter;
+    }
+
+    @Override
+    public IFrameReader createReceiveSideReader(HyracksContext ctx, JobPlan plan, IConnectionDemultiplexer demux, int index)
+            throws HyracksDataException {
+        IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+        for(int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        JobSpecification spec = plan.getJobSpecification();
+        return new SortMergeFrameReader(ctx, demux, sortFields, comparators, spec.getConnectorRecordDescriptor(this));
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MToNRangePartitioningConnectorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MToNRangePartitioningConnectorDescriptor.java
new file mode 100644
index 0000000..f455f01
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MToNRangePartitioningConnectorDescriptor.java
@@ -0,0 +1,133 @@
+/*
+ * 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.coreops;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.NonDeterministicFrameReader;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractConnectorDescriptor;
+
+public class MToNRangePartitioningConnectorDescriptor extends AbstractConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private class RangeDataWriter implements IFrameWriter {
+        private final int consumerPartitionCount;
+        private final IFrameWriter[] epWriters;
+        private final FrameTupleAppender[] appenders;
+        private final FrameTupleAccessor tupleAccessor;
+
+        public RangeDataWriter(HyracksContext ctx, int consumerPartitionCount, IFrameWriter[] epWriters,
+                FrameTupleAppender[] appenders, RecordDescriptor recordDescriptor) {
+            this.consumerPartitionCount = consumerPartitionCount;
+            this.epWriters = epWriters;
+            this.appenders = appenders;
+            tupleAccessor = new FrameTupleAccessor(ctx, recordDescriptor);
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            for (int i = 0; i < epWriters.length; ++i) {
+                if (appenders[i].getTupleCount() > 0) {
+                    flushFrame(appenders[i].getBuffer(), epWriters[i]);
+                }
+                epWriters[i].close();
+            }
+        }
+
+        private void flushFrame(ByteBuffer buffer, IFrameWriter frameWriter) throws HyracksDataException {
+            buffer.position(0);
+            buffer.limit(buffer.capacity());
+            frameWriter.nextFrame(buffer);
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            if (true) {
+                throw new UnsupportedOperationException();
+            }
+            tupleAccessor.reset(buffer);
+            int slotLength = tupleAccessor.getFieldSlotsLength();
+            int tupleCount = tupleAccessor.getTupleCount();
+            for (int i = 0; i < tupleCount; ++i) {
+                int startOffset = tupleAccessor.getTupleStartOffset(i);
+                int fStart = tupleAccessor.getFieldStartOffset(i, partitioningField);
+                int fEnd = tupleAccessor.getFieldEndOffset(i, partitioningField);
+                int h = 0;
+                FrameTupleAppender appender = appenders[h];
+                if (!appender.append(tupleAccessor, i)) {
+                    ByteBuffer appenderBuffer = appender.getBuffer();
+                    flushFrame(appenderBuffer, epWriters[h]);
+                    appender.reset(appenderBuffer, true);
+                }
+            }
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            for (int i = 0; i < epWriters.length; ++i) {
+                epWriters[i].open();
+                appenders[i].reset(appenders[i].getBuffer(), true);
+            }
+        }
+    }
+
+    private final int partitioningField;
+    private final Object[] splitVector;
+
+    public MToNRangePartitioningConnectorDescriptor(JobSpecification spec, int partitioningField, Object[] splitVector) {
+        super(spec);
+        this.partitioningField = partitioningField;
+        this.splitVector = splitVector;
+    }
+
+    @Override
+    public IFrameWriter createSendSideWriter(HyracksContext ctx, JobPlan plan, IEndpointDataWriterFactory edwFactory,
+            int index) throws HyracksDataException {
+        JobSpecification spec = plan.getJobSpecification();
+        final int consumerPartitionCount = spec.getConsumer(this).getPartitions().length;
+        final IFrameWriter[] epWriters = new IFrameWriter[consumerPartitionCount];
+        final FrameTupleAppender[] appenders = new FrameTupleAppender[consumerPartitionCount];
+        for (int i = 0; i < consumerPartitionCount; ++i) {
+            try {
+                epWriters[i] = edwFactory.createFrameWriter(i);
+                appenders[i] = new FrameTupleAppender(ctx);
+                appenders[i].reset(ctx.getResourceManager().allocateFrame(), true);
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        final RangeDataWriter rangeWriter = new RangeDataWriter(ctx, consumerPartitionCount, epWriters, appenders, spec
+                .getConnectorRecordDescriptor(this));
+        return rangeWriter;
+    }
+
+    @Override
+    public IFrameReader createReceiveSideReader(HyracksContext ctx, JobPlan plan, IConnectionDemultiplexer demux,
+            int index) throws HyracksDataException {
+        return new NonDeterministicFrameReader(ctx, demux);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MToNReplicatingConnectorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MToNReplicatingConnectorDescriptor.java
new file mode 100644
index 0000000..50d804a
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MToNReplicatingConnectorDescriptor.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.NonDeterministicFrameReader;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractConnectorDescriptor;
+
+public class MToNReplicatingConnectorDescriptor extends AbstractConnectorDescriptor {
+    public MToNReplicatingConnectorDescriptor(JobSpecification spec) {
+        super(spec);
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IFrameWriter createSendSideWriter(HyracksContext ctx, JobPlan plan, IEndpointDataWriterFactory edwFactory,
+            int index) throws HyracksDataException {
+        JobSpecification spec = plan.getJobSpecification();
+        final int consumerPartitionCount = spec.getConsumer(this).getPartitions().length;
+        final IFrameWriter[] epWriters = new IFrameWriter[consumerPartitionCount];
+        for (int i = 0; i < consumerPartitionCount; ++i) {
+            epWriters[i] = edwFactory.createFrameWriter(i);
+        }
+        return new IFrameWriter() {
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                buffer.mark();
+                for (int i = 0; i < epWriters.length; ++i) {
+                    if (i != 0) {
+                        buffer.reset();
+                    }
+                    epWriters[i].nextFrame(buffer);
+                }
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                for (int i = 0; i < epWriters.length; ++i) {
+                    epWriters[i].close();
+                }
+            }
+
+            @Override
+            public void open() throws HyracksDataException {
+                for (int i = 0; i < epWriters.length; ++i) {
+                    epWriters[i].open();
+                }
+            }
+        };
+    }
+
+    @Override
+    public IFrameReader createReceiveSideReader(HyracksContext ctx, JobPlan plan, IConnectionDemultiplexer demux,
+            int index) throws HyracksDataException {
+        return new NonDeterministicFrameReader(ctx, demux);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MapperOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MapperOperatorDescriptor.java
new file mode 100644
index 0000000..faad82c
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MapperOperatorDescriptor.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.coreops.util.DeserializedOperatorNodePushable;
+
+public class MapperOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private class MapperOperator implements IOpenableDataWriterOperator {
+        private IMapper mapper;
+        private IOpenableDataWriter<Object[]> writer;
+
+        @Override
+        public void close() throws HyracksDataException {
+            // writer.writeData(null);
+            writer.close();
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            mapper = mapperFactory.createMapper();
+            writer.open();
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            if (index != 0) {
+                throw new IllegalArgumentException();
+            }
+            this.writer = writer;
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            mapper.map(data, writer);
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    private final IMapperFactory mapperFactory;
+
+    public MapperOperatorDescriptor(JobSpecification spec, IMapperFactory mapperFactory,
+            RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.mapperFactory = mapperFactory;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        return new DeserializedOperatorNodePushable(ctx, new MapperOperator(), plan, getActivityNodeId());
+    }
+
+    @Override
+    public boolean supportsPullInterface() {
+        return false;
+    }
+
+    @Override
+    public boolean supportsPushInterface() {
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MaterializingOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MaterializingOperatorDescriptor.java
new file mode 100644
index 0000000..5e52989
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/MaterializingOperatorDescriptor.java
@@ -0,0 +1,214 @@
+/*
+ * 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.coreops;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.coreops.base.AbstractOperatorDescriptor;
+
+public class MaterializingOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+    protected static final String MATERIALIZED_FILE = "materialized-file";
+    protected static final String FRAME_COUNT = "frame-count";
+
+    public MaterializingOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        MaterializerActivityNode ma = new MaterializerActivityNode();
+        ReaderActivityNode ra = new ReaderActivityNode();
+
+        builder.addTask(ma);
+        builder.addSourceEdge(0, ma, 0);
+
+        builder.addTask(ra);
+        builder.addTargetEdge(0, ra, 0);
+
+        builder.addBlockingEdge(ma, ra);
+    }
+
+    private final class MaterializerActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+                int partition) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final HyracksContext ctx, JobPlan plan,
+                final IOperatorEnvironment env, int partition) {
+            return new IOperatorNodePushable() {
+                private FileChannel out;
+                private int frameCount;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    File outFile;
+                    try {
+                        outFile = ctx.getResourceManager().createFile("mat", ".dat");
+                        out = new RandomAccessFile(outFile, "rw").getChannel();
+                    } catch (IOException e) {
+                        throw new HyracksDataException(e);
+                    }
+                    env.set(MATERIALIZED_FILE, outFile.getAbsolutePath());
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    ++frameCount;
+                    buffer.position(0);
+                    buffer.limit(buffer.capacity());
+                    int rem = buffer.capacity();
+                    while (rem > 0) {
+                        int c;
+                        try {
+                            c = out.write(buffer);
+                        } catch (IOException e) {
+                            throw new HyracksDataException(e);
+                        }
+                        rem -= c;
+                    }
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    try {
+                        env.set(FRAME_COUNT, frameCount);
+                        out.close();
+                    } catch (IOException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer) {
+                    throw new IllegalArgumentException();
+                }
+            };
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return MaterializingOperatorDescriptor.this;
+        }
+
+        @Override
+        public boolean supportsPullInterface() {
+            return false;
+        }
+
+        @Override
+        public boolean supportsPushInterface() {
+            return true;
+        }
+    }
+
+    private final class ReaderActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+                int partition) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final HyracksContext ctx, JobPlan plan,
+                final IOperatorEnvironment env, int partition) {
+            return new IOperatorNodePushable() {
+                private IFrameWriter writer;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    try {
+                        File inFile = new File((String) env.get(MATERIALIZED_FILE));
+                        int frameCount = (Integer) env.get(FRAME_COUNT);
+                        FileChannel in = new RandomAccessFile(inFile, "r").getChannel();
+                        ByteBuffer frame = ctx.getResourceManager().allocateFrame();
+                        writer.open();
+                        for (int i = 0; i < frameCount; ++i) {
+                            frame.clear();
+                            int rem = frame.capacity();
+                            while (rem > 0) {
+                                int c = in.read(frame);
+                                rem -= c;
+                            }
+                            frame.flip();
+                            writer.nextFrame(frame);
+                        }
+                        writer.close();
+                        in.close();
+                    } catch (IOException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    throw new IllegalStateException();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    env.set(MATERIALIZED_FILE, null);
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer) {
+                    if (index != 0) {
+                        throw new IllegalArgumentException();
+                    }
+                    this.writer = writer;
+                }
+            };
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return MaterializingOperatorDescriptor.this;
+        }
+
+        @Override
+        public boolean supportsPullInterface() {
+            return false;
+        }
+
+        @Override
+        public boolean supportsPushInterface() {
+            return true;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/OneToOneConnectorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/OneToOneConnectorDescriptor.java
new file mode 100644
index 0000000..c46026a
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/OneToOneConnectorDescriptor.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops;
+
+import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.NonDeterministicFrameReader;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractConnectorDescriptor;
+
+public class OneToOneConnectorDescriptor extends AbstractConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public OneToOneConnectorDescriptor(JobSpecification spec) {
+        super(spec);
+    }
+
+    @Override
+    public IFrameWriter createSendSideWriter(HyracksContext ctx, JobPlan plan, IEndpointDataWriterFactory edwFactory,
+            int index) throws HyracksDataException {
+        return edwFactory.createFrameWriter(index);
+    }
+
+    @Override
+    public IFrameReader createReceiveSideReader(HyracksContext ctx, JobPlan plan, IConnectionDemultiplexer demux,
+            int index) throws HyracksDataException {
+        return new NonDeterministicFrameReader(ctx, demux);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/PrinterOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/PrinterOperatorDescriptor.java
new file mode 100644
index 0000000..fd32245
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/PrinterOperatorDescriptor.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.coreops.util.DeserializedOperatorNodePushable;
+
+public class PrinterOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public PrinterOperatorDescriptor(JobSpecification spec) {
+        super(spec, 1, 0);
+    }
+
+    private class PrinterOperator implements IOpenableDataWriterOperator {
+        @Override
+        public void open() throws HyracksDataException {
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            for (int i = 0; i < data.length; ++i) {
+                System.err.print(String.valueOf(data[i]));
+                System.err.print(", ");
+            }
+            System.err.println();
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            throw new IllegalArgumentException();
+        }
+    }
+
+    @Override
+    public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        return new DeserializedOperatorNodePushable(ctx, new PrinterOperator(), plan, getActivityNodeId());
+    }
+
+    @Override
+    public boolean supportsPullInterface() {
+        return false;
+    }
+
+    @Override
+    public boolean supportsPushInterface() {
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/SplitVectorOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/SplitVectorOperatorDescriptor.java
new file mode 100644
index 0000000..feb4d2f
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/SplitVectorOperatorDescriptor.java
@@ -0,0 +1,185 @@
+/*
+ * 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.coreops;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.coreops.base.AbstractOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.coreops.util.DeserializedOperatorNodePushable;
+
+public class SplitVectorOperatorDescriptor extends AbstractOperatorDescriptor {
+    private class CollectActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public ActivityNodeId getActivityNodeId() {
+            return id;
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return SplitVectorOperatorDescriptor.this;
+        }
+
+        @Override
+        public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+                int partition) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(HyracksContext ctx, JobPlan plan, final IOperatorEnvironment env,
+                int partition) {
+            IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
+                private ArrayList<Object[]> buffer;
+
+                @Override
+                public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+                    throw new IllegalArgumentException();
+                }
+
+                @Override
+                public void open() throws HyracksDataException {
+                    buffer = new ArrayList<Object[]>();
+                    env.set(BUFFER, buffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+
+                }
+
+                @Override
+                public void writeData(Object[] data) throws HyracksDataException {
+                    buffer.add(data);
+                }
+            };
+            return new DeserializedOperatorNodePushable(ctx, op, plan, getActivityNodeId());
+        }
+
+        @Override
+        public boolean supportsPullInterface() {
+            return false;
+        }
+
+        @Override
+        public boolean supportsPushInterface() {
+            return true;
+        }
+    }
+
+    private class SplitActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return SplitVectorOperatorDescriptor.this;
+        }
+
+        @Override
+        public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+                int partition) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(HyracksContext ctx, JobPlan plan, final IOperatorEnvironment env,
+                int partition) {
+            IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
+                private IOpenableDataWriter<Object[]> writer;
+
+                @Override
+                public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+                    if (index != 0) {
+                        throw new IllegalArgumentException();
+                    }
+                    this.writer = writer;
+                }
+
+                @Override
+                public void open() throws HyracksDataException {
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                }
+
+                @Override
+                public void writeData(Object[] data) throws HyracksDataException {
+                    List<Object[]> buffer = (List<Object[]>) env.get(BUFFER);
+                    int n = buffer.size();
+                    int step = (int) Math.floor(n / (float) splits);
+                    writer.open();
+                    for (int i = 0; i < splits; ++i) {
+                        writer.writeData(buffer.get(step * (i + 1) - 1));
+                    }
+                    writer.close();
+                }
+            };
+            return new DeserializedOperatorNodePushable(ctx, op, plan, getActivityNodeId());
+        }
+
+        @Override
+        public boolean supportsPullInterface() {
+            return false;
+        }
+
+        @Override
+        public boolean supportsPushInterface() {
+            return true;
+        }
+    }
+
+    private static final String BUFFER = "buffer";
+
+    private static final long serialVersionUID = 1L;
+
+    private final int splits;
+
+    public SplitVectorOperatorDescriptor(JobSpecification spec, int splits, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.splits = splits;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        CollectActivity ca = new CollectActivity();
+        SplitActivity sa = new SplitActivity();
+
+        builder.addTask(ca);
+        builder.addSourceEdge(0, ca, 0);
+
+        builder.addTask(sa);
+        builder.addTargetEdge(0, sa, 0);
+
+        builder.addBlockingEdge(ca, sa);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/aggregators/ReflectionBasedTupleAggregatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/aggregators/ReflectionBasedTupleAggregatorFactory.java
new file mode 100644
index 0000000..f649e48
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/aggregators/ReflectionBasedTupleAggregatorFactory.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.coreops.aggregators;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.coreops.ITupleAggregator;
+import edu.uci.ics.hyracks.coreops.ITupleAggregatorFactory;
+
+public class ReflectionBasedTupleAggregatorFactory implements
+        ITupleAggregatorFactory {
+
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
+    private final Class<? extends ITupleAggregator> aggregatorClass;
+
+    public ReflectionBasedTupleAggregatorFactory(
+            Class<? extends ITupleAggregator> aggregatorClass) {
+        this.aggregatorClass = aggregatorClass;
+    }
+
+    @Override
+    public ITupleAggregator createTupleAggregator() throws HyracksDataException {
+        try {
+            return aggregatorClass.newInstance();
+        } catch (InstantiationException e) {
+            throw new HyracksDataException(e);
+        } catch (IllegalAccessException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/aggregators/SumGroupAggregator.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/aggregators/SumGroupAggregator.java
new file mode 100644
index 0000000..f489a55
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/aggregators/SumGroupAggregator.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.aggregators;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.coreops.group.IGroupAggregator;
+
+public class SumGroupAggregator implements IGroupAggregator {
+    private static final long serialVersionUID = 1L;
+    private int keyIdx;
+
+    public SumGroupAggregator(int keyIdx) {
+        this.keyIdx = keyIdx;
+    }
+
+    @Override
+    public void aggregate(IDataReader<Object[]> reader, IDataWriter<Object[]> writer) throws HyracksDataException {
+        String key = "";
+        Object[] data;
+        int count = 0;
+        while ((data = reader.readData()) != null) {
+            key = (String) data[keyIdx];
+            ++count;
+        }
+        writer.writeData(new Object[] { key, count });
+    }
+
+    @Override
+    public void close() throws Exception {
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/aggregators/SumStringGroupAggregator.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/aggregators/SumStringGroupAggregator.java
new file mode 100644
index 0000000..7c774d3
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/aggregators/SumStringGroupAggregator.java
@@ -0,0 +1,45 @@
+/*
+ * 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.coreops.aggregators;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.coreops.group.IGroupAggregator;
+
+public class SumStringGroupAggregator implements IGroupAggregator {
+    private static final long serialVersionUID = 1L;
+    private int keyIdx;
+
+    public SumStringGroupAggregator(int keyIdx) {
+        this.keyIdx = keyIdx;
+    }
+
+    @Override
+    public void aggregate(IDataReader<Object[]> reader, IDataWriter<Object[]> writer) throws HyracksDataException {
+        String key = "";
+        Object[] data;
+        int count = 0;
+        while ((data = reader.readData()) != null) {
+            key = (String) data[keyIdx];
+            ++count;
+        }
+        writer.writeData(new Object[] { key, String.valueOf(count) });
+    }
+
+    @Override
+    public void close() throws Exception {
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/aggregators/SumTupleAggregator.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/aggregators/SumTupleAggregator.java
new file mode 100644
index 0000000..cfa512e
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/aggregators/SumTupleAggregator.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.coreops.aggregators;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.coreops.ITupleAggregator;
+
+public class SumTupleAggregator implements ITupleAggregator {
+
+    private Object key;
+    private int count;
+
+    @Override
+    public void add(Object[] data) {
+        count++;
+    }
+
+    @Override
+    public void init(Object[] data) {
+        key = data[0];
+        count = 0;
+    }
+
+    @Override
+    public void write(IDataWriter<Object[]> writer) throws HyracksDataException {
+        writer.writeData(new Object[] { key, count });
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/AbstractActivityNode.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/AbstractActivityNode.java
new file mode 100644
index 0000000..b2e39e8
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/AbstractActivityNode.java
@@ -0,0 +1,35 @@
+/*
+ * 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.coreops.base;
+
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
+
+public abstract class AbstractActivityNode implements IActivityNode {
+    private static final long serialVersionUID = 1L;
+
+    protected final ActivityNodeId id;
+
+    public AbstractActivityNode() {
+        this.id = new ActivityNodeId(getOwner().getOperatorId(), UUID.randomUUID());
+    }
+
+    @Override
+    public ActivityNodeId getActivityNodeId() {
+        return id;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/AbstractConnectorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/AbstractConnectorDescriptor.java
new file mode 100644
index 0000000..72dbcbf
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/AbstractConnectorDescriptor.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.base;
+
+import java.util.UUID;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public abstract class AbstractConnectorDescriptor implements IConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+    protected final ConnectorDescriptorId id;
+
+    public AbstractConnectorDescriptor(JobSpecification spec) {
+        this.id = new ConnectorDescriptorId(UUID.randomUUID());
+        spec.getConnectorMap().put(id, this);
+    }
+
+    public ConnectorDescriptorId getConnectorId() {
+        return id;
+    }
+
+    @Override
+    public JSONObject toJSON() throws JSONException {
+        JSONObject jconn = new JSONObject();
+
+        jconn.put("type", "connector");
+        jconn.put("id", getConnectorId().getId().toString());
+        jconn.put("java-class", getClass().getName());
+
+        return jconn;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/AbstractOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/AbstractOperatorDescriptor.java
new file mode 100644
index 0000000..b411020
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/AbstractOperatorDescriptor.java
@@ -0,0 +1,101 @@
+/*
+ * 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.coreops.base;
+
+import java.util.UUID;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public abstract class AbstractOperatorDescriptor implements IOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    protected final OperatorDescriptorId odId;
+
+    protected PartitionConstraint partitionConstraint;
+
+    protected String[] partitions;
+
+    protected final RecordDescriptor[] recordDescriptors;
+
+    protected final int inputArity;
+
+    protected final int outputArity;
+
+    public AbstractOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity) {
+        odId = new OperatorDescriptorId(UUID.randomUUID());
+        this.inputArity = inputArity;
+        this.outputArity = outputArity;
+        recordDescriptors = new RecordDescriptor[outputArity];
+        spec.getOperatorMap().put(getOperatorId(), this);
+    }
+
+    @Override
+    public final OperatorDescriptorId getOperatorId() {
+        return odId;
+    }
+
+    @Override
+    public int getInputArity() {
+        return inputArity;
+    }
+
+    @Override
+    public int getOutputArity() {
+        return outputArity;
+    }
+
+    @Override
+    public PartitionConstraint getPartitionConstraint() {
+        return partitionConstraint;
+    }
+
+    @Override
+    public void setPartitionConstraint(PartitionConstraint partitionConstraint) {
+        this.partitionConstraint = partitionConstraint;
+    }
+
+    @Override
+    public String[] getPartitions() {
+        return partitions;
+    }
+
+    @Override
+    public void setPartitions(String[] partitions) {
+        this.partitions = partitions;
+    }
+
+    @Override
+    public RecordDescriptor[] getOutputRecordDescriptors() {
+        return recordDescriptors;
+    }
+
+    @Override
+    public JSONObject toJSON() throws JSONException {
+        JSONObject jop = new JSONObject();
+        jop.put("type", "operator");
+        jop.put("id", getOperatorId().getId().toString());
+        jop.put("java-class", getClass().getName());
+        jop.put("in-arity", getInputArity());
+        jop.put("out-arity", getOutputArity());
+        return jop;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/AbstractSingleActivityOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/AbstractSingleActivityOperatorDescriptor.java
new file mode 100644
index 0000000..5843f10
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/AbstractSingleActivityOperatorDescriptor.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.coreops.base;
+
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public abstract class AbstractSingleActivityOperatorDescriptor extends AbstractOperatorDescriptor implements
+        IActivityNode {
+    private static final long serialVersionUID = 1L;
+
+    protected final ActivityNodeId activityNodeId;
+
+    public AbstractSingleActivityOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity) {
+        super(spec, inputArity, outputArity);
+        activityNodeId = new ActivityNodeId(odId, UUID.randomUUID());
+    }
+
+    @Override
+    public ActivityNodeId getActivityNodeId() {
+        return activityNodeId;
+    }
+
+    @Override
+    public final IOperatorDescriptor getOwner() {
+        return this;
+    }
+
+    @Override
+    public final void contributeTaskGraph(IActivityGraphBuilder builder) {
+        builder.addTask(this);
+        for (int i = 0; i < getInputArity(); ++i) {
+            builder.addSourceEdge(i, this, i);
+        }
+        for (int i = 0; i < getOutputArity(); ++i) {
+            builder.addTargetEdge(i, this, i);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/IOpenableDataWriterOperator.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/IOpenableDataWriterOperator.java
new file mode 100644
index 0000000..e6500d6
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/base/IOpenableDataWriterOperator.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.base;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+
+public interface IOpenableDataWriterOperator extends IOpenableDataWriter<Object[]> {
+    public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer);
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatBinaryComparatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatBinaryComparatorFactory.java
new file mode 100644
index 0000000..353d41d
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatBinaryComparatorFactory.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.coreops.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class FloatBinaryComparatorFactory implements IBinaryComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static final FloatBinaryComparatorFactory INSTANCE = new FloatBinaryComparatorFactory();
+
+    private FloatBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                return Float.compare(FloatSerializerDeserializer.getFloat(b1, s1), FloatSerializerDeserializer
+                        .getFloat(b2, s2));
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatBinaryHashFunctionFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..071ac70
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatBinaryHashFunctionFactory.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.coreops.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class FloatBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static final FloatBinaryHashFunctionFactory INSTANCE = new FloatBinaryHashFunctionFactory();
+
+    private FloatBinaryHashFunctionFactory() {
+    }
+
+    @Override
+    public IBinaryHashFunction createBinaryHashFunction() {
+        return new IBinaryHashFunction() {
+            @Override
+            public int hash(byte[] bytes, int offset, int length) {
+                return FloatSerializerDeserializer.getIntBits(bytes, offset);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatComparatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatComparatorFactory.java
new file mode 100644
index 0000000..9cbf499
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatComparatorFactory.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.coreops.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+
+public class FloatComparatorFactory implements IComparatorFactory<Float> {
+    private static final long serialVersionUID = 1L;
+
+    public static final FloatComparatorFactory INSTANCE = new FloatComparatorFactory();
+
+    private FloatComparatorFactory() {
+    }
+
+    @Override
+    public IComparator<Float> createComparator() {
+        return new IComparator<Float>() {
+            @Override
+            public int compare(Float o1, Float o2) {
+                return o1.compareTo(o2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatHashFunctionFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatHashFunctionFactory.java
new file mode 100644
index 0000000..a2831e8
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatHashFunctionFactory.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.coreops.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IHashFunctionFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class FloatHashFunctionFactory implements IHashFunctionFactory<Float> {
+    private static final long serialVersionUID = 1L;
+
+    public static final FloatHashFunctionFactory INSTANCE = new FloatHashFunctionFactory();
+
+    private FloatHashFunctionFactory() {
+    }
+
+    @Override
+    public IHashFunction<Float> createHashFunction() {
+        return new IHashFunction<Float>() {
+            @Override
+            public int hash(Float o) throws HyracksDataException {
+                return o.hashCode();
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatSerializerDeserializer.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatSerializerDeserializer.java
new file mode 100644
index 0000000..855da0e
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/FloatSerializerDeserializer.java
@@ -0,0 +1,57 @@
+/*
+ * 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.coreops.data;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class FloatSerializerDeserializer implements ISerializerDeserializer<Float> {
+    private static final long serialVersionUID = 1L;
+
+    public static final FloatSerializerDeserializer INSTANCE = new FloatSerializerDeserializer();
+
+    private FloatSerializerDeserializer() {
+    }
+
+    @Override
+    public Float deserialize(DataInput in) throws HyracksDataException {
+        try {
+            return in.readFloat();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void serialize(Float instance, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeFloat(instance.floatValue());
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    static float getFloat(byte[] bytes, int offset) {
+        return Float.intBitsToFloat(getIntBits(bytes, offset));
+    }
+
+    public static int getIntBits(byte[] bytes, int offset) {
+        return IntegerSerializerDeserializer.getInt(bytes, offset);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntArraySerializerDeserializer.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntArraySerializerDeserializer.java
new file mode 100644
index 0000000..410a6b9
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntArraySerializerDeserializer.java
@@ -0,0 +1,57 @@
+/*
+ * 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.coreops.data;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class IntArraySerializerDeserializer implements ISerializerDeserializer<int[]> {
+    private static final long serialVersionUID = 1L;
+
+    public static final IntArraySerializerDeserializer INSTANCE = new IntArraySerializerDeserializer();
+
+    private IntArraySerializerDeserializer() {
+    }
+
+    @Override
+    public int[] deserialize(DataInput in) throws HyracksDataException {
+        try {
+            int len = in.readInt();
+            int[] array = new int[len];
+            for (int i = 0; i < array.length; ++i) {
+                array[i] = in.readInt();
+            }
+            return array;
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void serialize(int[] instance, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeInt(instance.length);
+            for (int i = 0; i < instance.length; ++i) {
+                out.writeInt(instance[i]);
+            }
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerBinaryComparatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerBinaryComparatorFactory.java
new file mode 100644
index 0000000..f020909
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerBinaryComparatorFactory.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class IntegerBinaryComparatorFactory implements IBinaryComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static final IntegerBinaryComparatorFactory INSTANCE = new IntegerBinaryComparatorFactory();
+
+    private IntegerBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int v1 = IntegerSerializerDeserializer.getInt(b1, s1);
+                int v2 = IntegerSerializerDeserializer.getInt(b2, s2);
+                return v1 < v2 ? -1 : (v1 > v2 ? 1 : 0);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerBinaryHashFunctionFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..38b51a2
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerBinaryHashFunctionFactory.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.coreops.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class IntegerBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static final IntegerBinaryHashFunctionFactory INSTANCE = new IntegerBinaryHashFunctionFactory();
+
+    private IntegerBinaryHashFunctionFactory() {
+    }
+
+    @Override
+    public IBinaryHashFunction createBinaryHashFunction() {
+        return new IBinaryHashFunction() {
+            @Override
+            public int hash(byte[] bytes, int offset, int length) {
+                return IntegerSerializerDeserializer.getInt(bytes, offset);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerComparatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerComparatorFactory.java
new file mode 100644
index 0000000..9f4394b
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerComparatorFactory.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.coreops.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+
+public class IntegerComparatorFactory implements IComparatorFactory<Integer> {
+    private static final long serialVersionUID = 1L;
+
+    public static final IntegerComparatorFactory INSTANCE = new IntegerComparatorFactory();
+
+    private IntegerComparatorFactory() {
+    }
+
+    @Override
+    public IComparator<Integer> createComparator() {
+        return new IComparator<Integer>() {
+            @Override
+            public int compare(Integer o1, Integer o2) {
+                return o1.compareTo(o2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerHashFunctionFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerHashFunctionFactory.java
new file mode 100644
index 0000000..5785084
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerHashFunctionFactory.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.coreops.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IHashFunctionFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class IntegerHashFunctionFactory implements IHashFunctionFactory<Integer> {
+    private static final long serialVersionUID = 1L;
+
+    public static final IntegerHashFunctionFactory INSTANCE = new IntegerHashFunctionFactory();
+
+    private IntegerHashFunctionFactory() {
+    }
+
+    @Override
+    public IHashFunction<Integer> createHashFunction() {
+        return new IHashFunction<Integer>() {
+            @Override
+            public int hash(Integer o) throws HyracksDataException {
+                return o.hashCode();
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerSerializerDeserializer.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerSerializerDeserializer.java
new file mode 100644
index 0000000..74de8d3
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/IntegerSerializerDeserializer.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.coreops.data;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class IntegerSerializerDeserializer implements ISerializerDeserializer<Integer> {
+    private static final long serialVersionUID = 1L;
+
+    public static final IntegerSerializerDeserializer INSTANCE = new IntegerSerializerDeserializer();
+
+    private IntegerSerializerDeserializer() {
+    }
+
+    @Override
+    public Integer deserialize(DataInput in) throws HyracksDataException {
+        try {
+            return in.readInt();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void serialize(Integer instance, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeInt(instance.intValue());
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    static int getInt(byte[] bytes, int offset) {
+        return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16) + ((bytes[offset + 2] & 0xff) << 8)
+                + ((bytes[offset + 3] & 0xff) << 0);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringBinaryComparatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringBinaryComparatorFactory.java
new file mode 100644
index 0000000..54212b6
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringBinaryComparatorFactory.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.coreops.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class StringBinaryComparatorFactory implements IBinaryComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static final StringBinaryComparatorFactory INSTANCE = new StringBinaryComparatorFactory();
+
+    private StringBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int utflen1 = StringUtils.getUTFLen(b1, s1);
+                int utflen2 = StringUtils.getUTFLen(b2, s2);
+
+                int c1 = 0;
+                int c2 = 0;
+
+                int s1Start = s1 + 2;
+                int s2Start = s2 + 2;
+
+                while (c1 < utflen1 && c2 < utflen2) {
+                    char ch1 = StringUtils.charAt(b1, s1Start + c1);
+                    char ch2 = StringUtils.charAt(b2, s2Start + c2);
+
+                    if (ch1 != ch2) {
+                        return ch1 - ch2;
+                    }
+                    c1 += StringUtils.charSize(b1, s1Start + c1);
+                    c2 += StringUtils.charSize(b2, s2Start + c2);
+                }
+                return utflen1 - utflen2;
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringBinaryHashFunctionFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..3185442
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringBinaryHashFunctionFactory.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.coreops.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class StringBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+    public static final StringBinaryHashFunctionFactory INSTANCE = new StringBinaryHashFunctionFactory();
+
+    private static final long serialVersionUID = 1L;
+
+    private StringBinaryHashFunctionFactory() {
+    }
+
+    @Override
+    public IBinaryHashFunction createBinaryHashFunction() {
+        return new IBinaryHashFunction() {
+            @Override
+            public int hash(byte[] bytes, int offset, int length) {
+                int h = 0;
+                int utflen = StringUtils.getUTFLen(bytes, offset);
+                int sStart = offset + 2;
+                int c = 0;
+
+                while (c < utflen) {
+                    char ch = StringUtils.charAt(bytes, sStart + c);
+                    h = 31 * h + ch;
+                    c += StringUtils.charSize(bytes, sStart + c);
+                }
+                return h;
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringComparatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringComparatorFactory.java
new file mode 100644
index 0000000..d3cbbff
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringComparatorFactory.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.coreops.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+
+public class StringComparatorFactory implements IComparatorFactory<String> {
+    private static final long serialVersionUID = 1L;
+
+    public static final StringComparatorFactory INSTANCE = new StringComparatorFactory();
+
+    private StringComparatorFactory() {
+    }
+
+    @Override
+    public IComparator<String> createComparator() {
+        return new IComparator<String>() {
+            @Override
+            public int compare(String o1, String o2) {
+                return o1.compareTo(o2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringHashFunctionFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringHashFunctionFactory.java
new file mode 100644
index 0000000..c488138
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringHashFunctionFactory.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.coreops.data;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IHashFunctionFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class StringHashFunctionFactory implements IHashFunctionFactory<String> {
+    private static final long serialVersionUID = 1L;
+
+    public static final StringHashFunctionFactory INSTANCE = new StringHashFunctionFactory();
+
+    private StringHashFunctionFactory() {
+    }
+
+    @Override
+    public IHashFunction<String> createHashFunction() {
+        return new IHashFunction<String>() {
+            @Override
+            public int hash(String o) throws HyracksDataException {
+                return o.hashCode();
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringSerializerDeserializer.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringSerializerDeserializer.java
new file mode 100644
index 0000000..7342f1e
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringSerializerDeserializer.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.data;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class StringSerializerDeserializer implements ISerializerDeserializer<String> {
+    public static final StringSerializerDeserializer INSTANCE = new StringSerializerDeserializer();
+
+    private static final long serialVersionUID = 1L;
+
+    private StringSerializerDeserializer() {
+    }
+
+    @Override
+    public String deserialize(DataInput in) throws HyracksDataException {
+        try {
+            return in.readUTF();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void serialize(String instance, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeUTF(instance);
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringUtils.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringUtils.java
new file mode 100644
index 0000000..36a7ed6
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/data/StringUtils.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.data;
+
+public class StringUtils {
+    public static int charSize(byte[] b, int s) {
+        int c = (int) b[s] & 0xff;
+        switch (c >> 4) {
+            case 0:
+            case 1:
+            case 2:
+            case 3:
+            case 4:
+            case 5:
+            case 6:
+            case 7:
+                return 1;
+
+            case 12:
+            case 13:
+                return 2;
+
+            case 14:
+                return 3;
+        }
+        throw new IllegalStateException();
+    }
+
+    public static char charAt(byte[] b, int s) {
+        int c = (int) b[s] & 0xff;
+        switch (c >> 4) {
+            case 0:
+            case 1:
+            case 2:
+            case 3:
+            case 4:
+            case 5:
+            case 6:
+            case 7:
+                return (char) c;
+
+            case 12:
+            case 13:
+                return (char) (((c & 0x1F) << 6) | (((int) b[s + 1]) & 0x3F));
+
+            case 14:
+                return (char) (((c & 0x0F) << 12) | ((((int) b[s + 1]) & 0x3F) << 6) | ((((int) b[s + 2]) & 0x3F) << 0));
+
+            default:
+                throw new IllegalArgumentException();
+        }
+    }
+
+    public static int getUTFLen(byte[] b, int s) {
+        return ((b[s] & 0xff) << 8) + ((b[s + 1] & 0xff) << 0);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/AbstractFileScanOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/AbstractFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..82e805b
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/AbstractFileScanOperatorDescriptor.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.coreops.file;
+
+import java.io.File;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.coreops.util.DeserializedOperatorNodePushable;
+
+public abstract class AbstractFileScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    protected FileSplit[] splits;
+
+    public AbstractFileScanOperatorDescriptor(JobSpecification spec, FileSplit[] splits,
+            RecordDescriptor recordDescriptor) {
+        super(spec, 0, 1);
+        recordDescriptors[0] = recordDescriptor;
+        this.splits = splits;
+    }
+
+    protected abstract IRecordReader createRecordReader(File file, RecordDescriptor desc) throws Exception;
+
+    protected abstract void configure() throws Exception;
+
+    protected class FileScanOperator implements IOpenableDataWriterOperator {
+        private IOpenableDataWriter<Object[]> writer;
+        private int index;
+
+        FileScanOperator(int index) {
+            this.index = index;
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            if (index != 0) {
+                throw new IndexOutOfBoundsException("Invalid index: " + index);
+            }
+            this.writer = writer;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            FileSplit split = splits[index];
+            RecordDescriptor desc = recordDescriptors[0];
+            try {
+                IRecordReader reader = createRecordReader(split.getLocalFile(), desc);
+                if (desc == null) {
+                    desc = recordDescriptors[0];
+                }
+                writer.open();
+                try {
+                    while (true) {
+                        Object[] record = new Object[desc.getFields().length];
+                        if (!reader.read(record)) {
+                            break;
+                        }
+                        writer.writeData(record);
+                    }
+                } finally {
+                    reader.close();
+                    writer.close();
+                }
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            // do nothing
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+    }
+
+    @Override
+    public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        return new DeserializedOperatorNodePushable(ctx, new FileScanOperator(partition), plan, getActivityNodeId());
+    }
+
+    @Override
+    public boolean supportsPullInterface() {
+        return false;
+    }
+
+    @Override
+    public boolean supportsPushInterface() {
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/AbstractFileWriteOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/AbstractFileWriteOperatorDescriptor.java
new file mode 100644
index 0000000..5bf6a86
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/AbstractFileWriteOperatorDescriptor.java
@@ -0,0 +1,111 @@
+/*
+ * 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.coreops.file;
+
+import java.io.File;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.coreops.util.DeserializedOperatorNodePushable;
+
+public abstract class AbstractFileWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    protected class FileWriteOperator implements IOpenableDataWriterOperator {
+        private int index;
+        private IRecordWriter writer;
+
+        FileWriteOperator(int index) {
+            this.index = index;
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            throw new IllegalArgumentException();
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            FileSplit split = splits[index];
+            try {
+                writer = createRecordWriter(split.getLocalFile(), index);
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            writer.close();
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            try {
+                writer.write(data);
+
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    protected FileSplit[] splits;
+
+    public FileSplit[] getSplits() {
+		return splits;
+	}
+
+	public void setSplits(FileSplit[] splits) {
+		this.splits = splits;
+	}
+
+	public AbstractFileWriteOperatorDescriptor(JobSpecification spec, FileSplit[] splits) {
+        super(spec, 1, 0);
+        this.splits = splits;
+    }
+
+    protected abstract IRecordWriter createRecordWriter(File file, int index) throws Exception;
+
+    @Override
+    public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        return new DeserializedOperatorNodePushable(ctx, new FileWriteOperator(partition), plan, getActivityNodeId());
+    }
+
+    @Override
+    public boolean supportsPullInterface() {
+        return false;
+    }
+
+    @Override
+    public boolean supportsPushInterface() {
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/CSVFileScanOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/CSVFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..efcb8c8
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/CSVFileScanOperatorDescriptor.java
@@ -0,0 +1,129 @@
+/*
+ * 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.coreops.file;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class CSVFileScanOperatorDescriptor extends
+        AbstractFileScanOperatorDescriptor {
+    private static class CSVRecordReaderImpl implements IRecordReader {
+        private final BufferedReader in;
+        private final char separator;
+        private final String quotes;
+
+        CSVRecordReaderImpl(File file, RecordDescriptor desc, char separator,
+                String quotes) throws Exception {
+            in = new BufferedReader(new InputStreamReader(new FileInputStream(
+                    file)));
+            this.separator = separator;
+            this.quotes = quotes;
+        }
+
+        @Override
+        public void close() {
+            try {
+                in.close();
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+
+        @Override
+        public boolean read(Object[] record) throws Exception {
+            String line = in.readLine();
+            if (line == null) {
+                return false;
+            }
+            int fid = 0;
+            char[] chars = line.toCharArray();
+            int i = 0;
+            boolean insideQuote = false;
+            char quoteChar = 0;
+            int partStart = 0;
+            boolean skipNext = false;
+            while (fid < record.length && i < chars.length) {
+                char c = chars[i];
+                if (!skipNext) {
+                    if (quotes.indexOf(c) >= 0) {
+                        if (insideQuote) {
+                            if (quoteChar == c) {
+                                insideQuote = false;
+                            }
+                        } else {
+                            insideQuote = true;
+                            quoteChar = c;
+                        }
+                    } else if (c == separator) {
+                        if (!insideQuote) {
+                            record[fid++] = String.valueOf(chars, partStart, i
+                                    - partStart);
+                            partStart = i + 1;
+                        }
+                    } else if (c == '\\') {
+                        skipNext = true;
+                    }
+                } else {
+                    skipNext = false;
+                }
+                ++i;
+            }
+            if (fid < record.length) {
+                record[fid] = String.valueOf(chars, partStart, i - partStart);
+            }
+            return true;
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    private final char separator;
+    private final String quotes;
+
+    public CSVFileScanOperatorDescriptor(JobSpecification spec,
+            FileSplit[] splits, RecordDescriptor recordDescriptor) {
+        this(spec, splits, recordDescriptor, ',', "'\"");
+    }
+
+    public CSVFileScanOperatorDescriptor(JobSpecification spec,
+            FileSplit[] splits, RecordDescriptor recordDescriptor,
+            char separator, String quotes) {
+        super(spec, splits, recordDescriptor);
+        this.separator = separator;
+        this.quotes = quotes;
+    }
+
+    @Override
+    protected IRecordReader createRecordReader(File file, RecordDescriptor desc)
+            throws Exception {
+        return new CSVRecordReaderImpl(file, desc, separator, quotes);
+    }
+
+	@Override
+	protected void configure() throws Exception {
+		// currently a no-op, but is meant to initialize , if required before it is asked 
+		// to create a record reader
+		// this is executed at the node and is useful for operators that could not be 
+		// initialized from the client completely, because of lack of information specific 
+		// to the node where the operator gets executed. 
+		
+	}
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/FileSplit.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/FileSplit.java
new file mode 100644
index 0000000..361afca
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/FileSplit.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.file;
+
+import java.io.File;
+import java.io.Serializable;
+
+public class FileSplit implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final String nodeName;
+
+    private final File file;
+
+    public FileSplit(String nodeName, File file) {
+        this.nodeName = nodeName;
+        this.file = file;
+    }
+
+    public String getNodeName() {
+        return nodeName;
+    }
+
+    public File getLocalFile() {
+        return file;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/IRecordReader.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/IRecordReader.java
new file mode 100644
index 0000000..607c637
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/IRecordReader.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.file;
+
+public interface IRecordReader {
+
+	  public boolean read(Object[] record) throws Exception;
+
+      public void close();
+      
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/IRecordWriter.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/IRecordWriter.java
new file mode 100644
index 0000000..a270f86
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/IRecordWriter.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.coreops.file;
+
+import java.io.File;
+
+public interface IRecordWriter {
+	  
+      public void close();
+      
+      public void write(Object[] record) throws Exception;
+
+	
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/LineFileScanOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/LineFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..c2e5a3b
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/LineFileScanOperatorDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.file;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class LineFileScanOperatorDescriptor extends
+        AbstractFileScanOperatorDescriptor {
+    private static class LineReaderImpl extends  RecordReader {
+        private File file;
+
+        LineReaderImpl(File file) throws Exception {
+        	super(new Object[]{file});
+        	this.file = file;
+        }
+
+		@Override
+		public InputStream createInputStream(Object[] args) throws Exception{
+			this.file = (File)args[0];
+			return new FileInputStream(file) ;
+		}
+       }
+
+    private static final long serialVersionUID = 1L;
+
+    public LineFileScanOperatorDescriptor(JobSpecification spec,
+            FileSplit[] splits, RecordDescriptor recordDescriptor) {
+        super(spec, splits, recordDescriptor);
+    }
+
+    @Override
+    protected IRecordReader createRecordReader(File file, RecordDescriptor desc)
+            throws Exception {
+        return new LineReaderImpl(file);
+    }
+    
+    @Override
+	protected void configure() throws Exception {
+		// currently a no-op, but is meant to initialize , if required before it is asked 
+		// to create a record reader
+		// this is executed at the node and is useful for operators that could not be 
+		// initialized from the client completely, because of lack of information specific 
+		// to the node where the operator gets executed. 
+		
+	}
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/LineFileWriteOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/LineFileWriteOperatorDescriptor.java
new file mode 100644
index 0000000..5e7bfef
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/LineFileWriteOperatorDescriptor.java
@@ -0,0 +1,71 @@
+/*
+ * 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.coreops.file;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class LineFileWriteOperatorDescriptor extends AbstractFileWriteOperatorDescriptor {
+   
+	private static class LineWriterImpl extends RecordWriter {
+		
+		File file;
+		
+        LineWriterImpl(File file,int []columns, char separator) throws Exception {
+        	super(columns,separator, new Object[]{file});
+        	this.file=file;    
+        }
+
+	    private static final long serialVersionUID = 1L;
+	
+		@Override
+		public OutputStream createOutputStream(Object[] args) throws Exception {
+			return new FileOutputStream((File)args[0]);
+		}
+	}
+    
+    private int[]columns ;
+    private char separator;
+    
+    
+	public LineFileWriteOperatorDescriptor(JobSpecification spec,
+            FileSplit[] splits) {
+        this(spec, splits, null, RecordWriter.COMMA);
+    }
+
+    public LineFileWriteOperatorDescriptor(JobSpecification spec,
+            FileSplit[] splits, int[] columns) {
+        this(spec, splits, columns, RecordWriter.COMMA);
+    }
+
+    public LineFileWriteOperatorDescriptor(JobSpecification spec,
+            FileSplit[] splits, int[] columns, char separator) {
+        super(spec, splits);
+        this.columns = columns;
+        this.separator = separator;
+    }
+	
+	
+    @Override
+    protected IRecordWriter createRecordWriter(File file,int index) throws Exception {
+        return new LineWriterImpl(file,columns,separator);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/RecordFileScanOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/RecordFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..9df04f9
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/RecordFileScanOperatorDescriptor.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.file;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class RecordFileScanOperatorDescriptor extends AbstractFileScanOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public RecordFileScanOperatorDescriptor(JobSpecification spec, FileSplit[] splits, RecordDescriptor recordDescriptor) {
+        super(spec, splits, recordDescriptor);
+    }
+
+    private static class RecordReaderImpl implements IRecordReader {
+        private RecordDescriptor recordDesc;
+        private DataInputStream in;
+
+        RecordReaderImpl(File file, RecordDescriptor recordDesc) throws Exception {
+            this.in = new DataInputStream(new BufferedInputStream(new FileInputStream(file)));
+            this.recordDesc = recordDesc;
+        }
+
+        @Override
+        public boolean read(Object[] record) throws Exception {
+            in.mark(1);
+            if (in.read() < 0) {
+                return false;
+            }
+            in.reset();
+            for (int i = 0; i < record.length; ++i) {
+                record[i] = recordDesc.getFields()[i].deserialize(in);
+            }
+            return true;
+        }
+
+        @Override
+        public void close() {
+            try {
+                in.close();
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    @Override
+    protected IRecordReader createRecordReader(File file, RecordDescriptor desc) throws Exception {
+        return new RecordReaderImpl(file, desc);
+    }
+    
+    @Override
+	protected void configure() throws Exception {
+		// currently a no-op, but is meant to initialize , if required before it is asked 
+		// to create a record reader
+		// this is executed at the node and is useful for operators that could not be 
+		// initialized from the client completely, because of lack of information specific 
+		// to the node where the operator gets executed. 
+		
+	}
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/RecordReader.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/RecordReader.java
new file mode 100644
index 0000000..63f8f7e
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/RecordReader.java
@@ -0,0 +1,57 @@
+/*
+ * 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.coreops.file;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+public  abstract class RecordReader implements IRecordReader {
+
+    private final BufferedReader bufferedReader;
+	private InputStream inputStream;
+	
+	@Override
+	public void close() {
+        try {
+        	bufferedReader.close();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+
+    @Override
+    public boolean read(Object[] record) throws Exception {
+        String line = bufferedReader.readLine();
+        if (line == null) {
+            return false;
+        }
+        record[0] = line;
+        return true;
+    }
+	
+	public abstract InputStream createInputStream(Object[] args) throws Exception;
+	
+	public RecordReader(Object[] args) throws Exception{
+		try{
+			bufferedReader = new BufferedReader(new InputStreamReader(createInputStream(args)));
+		}catch(Exception e){
+			e.printStackTrace();
+			throw e;
+		}
+	}
+	
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/RecordWriter.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/RecordWriter.java
new file mode 100644
index 0000000..f92686d
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/file/RecordWriter.java
@@ -0,0 +1,86 @@
+/*
+ * 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.coreops.file;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+
+public abstract class RecordWriter implements IRecordWriter{
+
+	 
+	protected final BufferedWriter bufferedWriter;
+    protected final int[] columns;
+    protected final char separator;
+    
+    public static final char COMMA = ',';
+    
+    public RecordWriter(Object [] args) throws Exception{
+    	OutputStream outputStream = createOutputStream(args);
+    	if(outputStream != null){
+    		bufferedWriter = new BufferedWriter(new OutputStreamWriter(createOutputStream(args)));
+    	}else{
+    		bufferedWriter = null;
+    	}
+    	this.columns = null;
+    	this.separator = COMMA;
+    }
+    
+    public RecordWriter(int []columns, char separator, Object[] args) throws Exception{
+    	OutputStream outputStream = createOutputStream(args);
+    	if(outputStream != null){
+    		bufferedWriter = new BufferedWriter(new OutputStreamWriter(outputStream));
+    	}else{
+    		bufferedWriter = null;
+    	}
+    	this.columns = columns;
+    	this.separator = separator;
+    }
+    
+	@Override
+     public void close() {
+         try {
+             bufferedWriter.close();
+         } catch (IOException e) {
+             e.printStackTrace();
+         }
+     }
+
+     @Override
+     public void write(Object[] record) throws Exception {
+         if (columns == null) {
+             for (int i = 0; i < record.length; ++i) {
+                 if (i != 0) {
+                     bufferedWriter.write(separator);
+                 }
+                 bufferedWriter.write(String.valueOf(record[i]));
+             }
+         } else {
+             for (int i = 0; i < columns.length; ++i) {
+                 if (i != 0) {
+                     bufferedWriter.write(separator);
+                 }
+                 bufferedWriter.write(String.valueOf(record[columns[i]]));
+             }
+         }
+         bufferedWriter.write("\n");
+     }
+     
+     public abstract OutputStream createOutputStream(Object[] args) throws Exception;
+   
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/GroupingHashTable.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/GroupingHashTable.java
new file mode 100644
index 0000000..86c247d
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/GroupingHashTable.java
@@ -0,0 +1,194 @@
+/*
+ * 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.coreops.group;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.comm.io.FrameTuplePairComparator;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+class GroupingHashTable {
+    /**
+     * The pointers in the link store 3 int values for each entry in the
+     * hashtable: (bufferIdx, tIndex, accumulatorIdx).
+     * 
+     * @author vinayakb
+     */
+    private static class Link {
+        private static final int INIT_POINTERS_SIZE = 9;
+
+        int[] pointers;
+        int size;
+
+        Link() {
+            pointers = new int[INIT_POINTERS_SIZE];
+            size = 0;
+        }
+
+        void add(int bufferIdx, int tIndex, int accumulatorIdx) {
+            while (size + 3 > pointers.length) {
+                pointers = Arrays.copyOf(pointers, pointers.length * 2);
+            }
+            pointers[size++] = bufferIdx;
+            pointers[size++] = tIndex;
+            pointers[size++] = accumulatorIdx;
+        }
+    }
+
+    private static final int INIT_ACCUMULATORS_SIZE = 8;
+    private final HyracksContext ctx;
+    private final FrameTupleAppender appender;
+    private final List<ByteBuffer> buffers;
+    private final Link[] table;
+    private IAccumulatingAggregator[] accumulators;
+    private int accumulatorSize;
+
+    private int lastBIndex;
+    private final int[] fields;
+    private final int[] storedKeys;
+    private final IBinaryComparator[] comparators;
+    private final FrameTuplePairComparator ftpc;
+    private final ITuplePartitionComputer tpc;
+    private final IAccumulatingAggregatorFactory aggregatorFactory;
+    private final RecordDescriptor inRecordDescriptor;
+    private final RecordDescriptor outRecordDescriptor;
+
+    private final FrameTupleAccessor storedKeysAccessor;
+
+    GroupingHashTable(HyracksContext ctx, int[] fields, IBinaryComparatorFactory[] comparatorFactories,
+            ITuplePartitionComputerFactory tpcf, IAccumulatingAggregatorFactory aggregatorFactory,
+            RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor, int tableSize) {
+        this.ctx = ctx;
+        appender = new FrameTupleAppender(ctx);
+        buffers = new ArrayList<ByteBuffer>();
+        table = new Link[tableSize];
+        accumulators = new IAccumulatingAggregator[INIT_ACCUMULATORS_SIZE];
+        accumulatorSize = 0;
+        this.fields = fields;
+        storedKeys = new int[fields.length];
+        ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[fields.length];
+        for (int i = 0; i < fields.length; ++i) {
+            storedKeys[i] = i;
+            storedKeySerDeser[i] = inRecordDescriptor.getFields()[fields[i]];
+        }
+        comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        ftpc = new FrameTuplePairComparator(fields, storedKeys, comparators);
+        tpc = tpcf.createPartitioner();
+        this.aggregatorFactory = aggregatorFactory;
+        this.inRecordDescriptor = inRecordDescriptor;
+        this.outRecordDescriptor = outRecordDescriptor;
+        RecordDescriptor storedKeysRecordDescriptor = new RecordDescriptor(storedKeySerDeser);
+        storedKeysAccessor = new FrameTupleAccessor(ctx, storedKeysRecordDescriptor);
+        lastBIndex = -1;
+        addNewBuffer();
+    }
+
+    private void addNewBuffer() {
+        ByteBuffer buffer = ctx.getResourceManager().allocateFrame();
+        buffer.position(0);
+        buffer.limit(buffer.capacity());
+        buffers.add(buffer);
+        appender.reset(buffer, true);
+        ++lastBIndex;
+    }
+
+    private void flushFrame(FrameTupleAppender appender, IFrameWriter writer) throws HyracksDataException {
+        ByteBuffer frame = appender.getBuffer();
+        frame.position(0);
+        frame.limit(frame.capacity());
+        writer.nextFrame(appender.getBuffer());
+        appender.reset(appender.getBuffer(), true);
+    }
+
+    void insert(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+        int entry = tpc.partition(accessor, tIndex, table.length);
+        Link link = table[entry];
+        if (link == null) {
+            link = table[entry] = new Link();
+        }
+        IAccumulatingAggregator aggregator = null;
+        for (int i = 0; i < link.size; i += 3) {
+            int sbIndex = link.pointers[i];
+            int stIndex = link.pointers[i + 1];
+            int saIndex = link.pointers[i + 2];
+            storedKeysAccessor.reset(buffers.get(sbIndex));
+            int c = ftpc.compare(accessor, tIndex, storedKeysAccessor, stIndex);
+            if (c == 0) {
+                aggregator = accumulators[saIndex];
+                break;
+            }
+        }
+        if (aggregator == null) {
+            // Did not find the key. Insert a new entry.
+            if (!appender.appendProjection(accessor, tIndex, fields)) {
+                addNewBuffer();
+                if (!appender.appendProjection(accessor, tIndex, fields)) {
+                    throw new IllegalStateException();
+                }
+            }
+            int sbIndex = lastBIndex;
+            int stIndex = appender.getTupleCount() - 1;
+            if (accumulatorSize >= accumulators.length) {
+                accumulators = Arrays.copyOf(accumulators, accumulators.length * 2);
+            }
+            int saIndex = accumulatorSize++;
+            aggregator = accumulators[saIndex] = aggregatorFactory.createAggregator(inRecordDescriptor,
+                    outRecordDescriptor);
+            aggregator.init(accessor, tIndex);
+            link.add(sbIndex, stIndex, saIndex);
+        }
+        aggregator.accumulate(accessor, tIndex);
+    }
+
+    void write(IFrameWriter writer) throws HyracksDataException {
+        ByteBuffer buffer = ctx.getResourceManager().allocateFrame();
+        appender.reset(buffer, true);
+        for (int i = 0; i < table.length; ++i) {
+            Link link = table[i];
+            if (link != null) {
+                for (int j = 0; j < link.size; j += 3) {
+                    int bIndex = link.pointers[j];
+                    int tIndex = link.pointers[j + 1];
+                    int aIndex = link.pointers[j + 2];
+                    ByteBuffer keyBuffer = buffers.get(bIndex);
+                    storedKeysAccessor.reset(keyBuffer);
+                    IAccumulatingAggregator aggregator = accumulators[aIndex];
+                    while (!aggregator.output(appender, storedKeysAccessor, tIndex)) {
+                        flushFrame(appender, writer);
+                    }
+                }
+            }
+        }
+        if (appender.getTupleCount() != 0) {
+            flushFrame(appender, writer);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/HashGroupOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/HashGroupOperatorDescriptor.java
new file mode 100644
index 0000000..2fc6b63
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/HashGroupOperatorDescriptor.java
@@ -0,0 +1,192 @@
+/*
+ * 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.coreops.group;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.coreops.base.AbstractOperatorDescriptor;
+
+public class HashGroupOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final String HASHTABLE = "hashtable";
+
+    private static final long serialVersionUID = 1L;
+
+    private final int[] keys;
+    private final ITuplePartitionComputerFactory tpcf;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final IAccumulatingAggregatorFactory aggregatorFactory;
+    private final int tableSize;
+
+    public HashGroupOperatorDescriptor(JobSpecification spec, int[] keys, ITuplePartitionComputerFactory tpcf,
+            IBinaryComparatorFactory[] comparatorFactories, IAccumulatingAggregatorFactory aggregatorFactory,
+            RecordDescriptor recordDescriptor, int tableSize) {
+        super(spec, 1, 1);
+        this.keys = keys;
+        this.tpcf = tpcf;
+        this.comparatorFactories = comparatorFactories;
+        this.aggregatorFactory = aggregatorFactory;
+        recordDescriptors[0] = recordDescriptor;
+        this.tableSize = tableSize;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        HashBuildActivity ha = new HashBuildActivity();
+        builder.addTask(ha);
+
+        OutputActivity oa = new OutputActivity();
+        builder.addTask(oa);
+
+        builder.addSourceEdge(0, ha, 0);
+        builder.addTargetEdge(0, oa, 0);
+        builder.addBlockingEdge(ha, oa);
+    }
+
+    private class HashBuildActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+                int partition) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final HyracksContext ctx, JobPlan plan,
+                final IOperatorEnvironment env, int partition) {
+            final RecordDescriptor rd0 = plan.getJobSpecification()
+                    .getOperatorInputRecordDescriptor(getOperatorId(), 0);
+            final FrameTupleAccessor accessor = new FrameTupleAccessor(ctx, rd0);
+            return new IOperatorNodePushable() {
+                private GroupingHashTable table;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    table = new GroupingHashTable(ctx, keys, comparatorFactories, tpcf, aggregatorFactory, rd0,
+                            recordDescriptors[0], tableSize);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    accessor.reset(buffer);
+                    int tupleCount = accessor.getTupleCount();
+                    for (int i = 0; i < tupleCount; ++i) {
+                        table.insert(accessor, i);
+                    }
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    env.set(HASHTABLE, table);
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer) {
+                    throw new IllegalArgumentException();
+                }
+            };
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return HashGroupOperatorDescriptor.this;
+        }
+
+        @Override
+        public boolean supportsPullInterface() {
+            return false;
+        }
+
+        @Override
+        public boolean supportsPushInterface() {
+            return true;
+        }
+    }
+
+    private class OutputActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+                int partition) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(HyracksContext ctx, JobPlan plan, final IOperatorEnvironment env,
+                int partition) {
+            return new IOperatorNodePushable() {
+                private IFrameWriter writer;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    GroupingHashTable table = (GroupingHashTable) env.get(HASHTABLE);
+                    writer.open();
+                    table.write(writer);
+                    writer.close();
+                    env.set(HASHTABLE, null);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    throw new IllegalStateException();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    // do nothing
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer) {
+                    if (index != 0) {
+                        throw new IllegalArgumentException();
+                    }
+                    this.writer = writer;
+                }
+            };
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return HashGroupOperatorDescriptor.this;
+        }
+
+        @Override
+        public boolean supportsPullInterface() {
+            return false;
+        }
+
+        @Override
+        public boolean supportsPushInterface() {
+            return true;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/IAccumulatingAggregator.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/IAccumulatingAggregator.java
new file mode 100644
index 0000000..d9e9337
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/IAccumulatingAggregator.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.coreops.group;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+
+public interface IAccumulatingAggregator {
+    /**
+     * Called once per aggregator before calling accumulate for the first time.
+     * 
+     * @param accessor
+     *            - Accessor to the data tuple.
+     * @param tIndex
+     *            - Index of the tuple in the accessor.
+     * @throws HyracksDataException
+     */
+    public void init(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException;
+
+    /**
+     * Called once per tuple that belongs to this group.
+     * 
+     * @param accessor
+     *            - Accessor to data tuple.
+     * @param tIndex
+     *            - Index of tuple in the accessor.
+     * @throws HyracksDataException
+     */
+    public void accumulate(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException;
+
+    /**
+     * Called finally to emit output. This method is called until it returns true. The method is free to
+     * write out output to the provided appender until there is no more space and return false. It is the
+     * caller's responsibility to flush and make room in the appender before this method is called again.
+     * 
+     * @param appender
+     *            - Appender to write output to.
+     * @param accessor
+     *            - Accessor to access the key.
+     * @param tIndex
+     *            - Tuple index of the key in the accessor.
+     * @return true if all output is written, false if the appender is full.
+     * @throws HyracksDataException
+     */
+    public boolean output(FrameTupleAppender appender, FrameTupleAccessor accessor, int tIndex)
+            throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/IAccumulatingAggregatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/IAccumulatingAggregatorFactory.java
new file mode 100644
index 0000000..e5c935c
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/IAccumulatingAggregatorFactory.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.group;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public interface IAccumulatingAggregatorFactory extends Serializable {
+    IAccumulatingAggregator createAggregator(RecordDescriptor inRecordDesc, RecordDescriptor outRecordDescriptor);
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/IGroupAggregator.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/IGroupAggregator.java
new file mode 100644
index 0000000..69a0f47
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/IGroupAggregator.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.group;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IGroupAggregator extends Serializable {
+    public void aggregate(IDataReader<Object[]> reader, IDataWriter<Object[]> writer) throws HyracksDataException;
+
+    public void close() throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/PreclusteredGroupOperator.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/PreclusteredGroupOperator.java
new file mode 100644
index 0000000..4af62dc
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/PreclusteredGroupOperator.java
@@ -0,0 +1,113 @@
+/*
+ * 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.coreops.group;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.coreops.base.IOpenableDataWriterOperator;
+
+public class PreclusteredGroupOperator implements IOpenableDataWriterOperator {
+    private final int[] groupFields;
+
+    private final IComparator[] comparators;
+
+    private final IGroupAggregator aggregator;
+
+    private Object[] lastData;
+
+    private IOpenableDataWriter<Object[]> writer;
+
+    private List<Object[]> buffer;
+
+    private IOpenableDataReader<Object[]> reader;
+
+    public PreclusteredGroupOperator(int[] groupFields, IComparator[] comparators, IGroupAggregator aggregator) {
+        this.groupFields = groupFields;
+        this.comparators = comparators;
+        this.aggregator = aggregator;
+        buffer = new ArrayList<Object[]>();
+        reader = new IOpenableDataReader<Object[]>() {
+            private int idx;
+
+            @Override
+            public void open() {
+                idx = 0;
+            }
+
+            @Override
+            public void close() {
+            }
+
+            @Override
+            public Object[] readData() {
+                return idx >= buffer.size() ? null : buffer.get(idx++);
+            }
+        };
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (!buffer.isEmpty()) {
+            aggregate();
+        }
+        writer.close();
+    }
+
+    private void aggregate() throws HyracksDataException {
+        reader.open();
+        aggregator.aggregate(reader, writer);
+        reader.close();
+        buffer.clear();
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        lastData = null;
+        writer.open();
+    }
+
+    @Override
+    public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+        if (index != 0) {
+            throw new IllegalArgumentException();
+        }
+        this.writer = writer;
+    }
+
+    @Override
+    public void writeData(Object[] data) throws HyracksDataException {
+        if (lastData != null && compare(data, lastData) != 0) {
+            aggregate();
+        }
+        lastData = data;
+        buffer.add(data);
+    }
+
+    private int compare(Object[] d1, Object[] d2) {
+        for (int i = 0; i < groupFields.length; ++i) {
+            int fIdx = groupFields[i];
+            int c = comparators[i].compare(d1[fIdx], d2[fIdx]);
+            if (c != 0) {
+                return c;
+            }
+        }
+        return 0;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/PreclusteredGroupOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/PreclusteredGroupOperatorDescriptor.java
new file mode 100644
index 0000000..0232d73
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/group/PreclusteredGroupOperatorDescriptor.java
@@ -0,0 +1,71 @@
+/*
+ * 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.coreops.group;
+
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.util.DeserializedOperatorNodePushable;
+
+public class PreclusteredGroupOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private final int[] groupFields;
+    private final IGroupAggregator aggregator;
+    private final IComparatorFactory[] comparatorFactories;
+
+    private static final long serialVersionUID = 1L;
+
+    public PreclusteredGroupOperatorDescriptor(JobSpecification spec, int[] groupFields,
+            IComparatorFactory[] comparatorFactories, IGroupAggregator aggregator, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.groupFields = groupFields;
+        this.comparatorFactories = comparatorFactories;
+        this.aggregator = aggregator;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        IComparator[] comparators = new IComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createComparator();
+        }
+        return new DeserializedOperatorNodePushable(ctx, new PreclusteredGroupOperator(groupFields, comparators,
+                aggregator), plan, getActivityNodeId());
+    }
+
+    @Override
+    public boolean supportsPullInterface() {
+        return false;
+    }
+
+    @Override
+    public boolean supportsPushInterface() {
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/AbstractHadoopFileScanOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/AbstractHadoopFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..5ee84e4
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/AbstractHadoopFileScanOperatorDescriptor.java
@@ -0,0 +1,165 @@
+/*
+ * 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.coreops.hadoop;
+
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.Counters.Counter;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.coreops.file.IRecordReader;
+import edu.uci.ics.hyracks.coreops.util.DeserializedOperatorNodePushable;
+import edu.uci.ics.hyracks.hadoop.util.HadoopFileSplit;
+
+public abstract class AbstractHadoopFileScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+    protected HadoopFileSplit[] splits;
+
+    public AbstractHadoopFileScanOperatorDescriptor(JobSpecification spec, HadoopFileSplit[] splits,
+            RecordDescriptor recordDescriptor) {
+        super(spec, 0, 1);
+        recordDescriptors[0] = recordDescriptor;
+        this.splits = splits;
+    }
+
+    protected abstract IRecordReader createRecordReader(HadoopFileSplit fileSplit, RecordDescriptor desc)
+            throws Exception;
+
+    protected class FileScanOperator implements IOpenableDataWriterOperator {
+        private IOpenableDataWriter<Object[]> writer;
+        private int index;
+
+        FileScanOperator(int index) {
+            this.index = index;
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            if (index != 0) {
+                throw new IndexOutOfBoundsException("Invalid index: " + index);
+            }
+            this.writer = writer;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            HadoopFileSplit split = splits[index];
+            RecordDescriptor desc = recordDescriptors[0];
+            try {
+                IRecordReader reader = createRecordReader(split, desc);
+                if (desc == null) {
+                    desc = recordDescriptors[0];
+                }
+                writer.open();
+                try {
+                    while (true) {
+                        Object[] record = new Object[desc.getFields().length];
+                        if (!reader.read(record)) {
+                            break;
+                        }
+                        writer.writeData(record);
+                    }
+                } finally {
+                    reader.close();
+                    writer.close();
+                }
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            // do nothing
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+    }
+
+    protected Reporter createReporter() {
+        return new Reporter() {
+            @Override
+            public Counter getCounter(Enum<?> name) {
+                return null;
+            }
+
+            @Override
+            public Counter getCounter(String group, String name) {
+                return null;
+            }
+
+            @Override
+            public InputSplit getInputSplit() throws UnsupportedOperationException {
+                return null;
+            }
+
+            @Override
+            public void incrCounter(Enum<?> key, long amount) {
+
+            }
+
+            @Override
+            public void incrCounter(String group, String counter, long amount) {
+
+            }
+
+            @Override
+            public void progress() {
+
+            }
+
+            @Override
+            public void setStatus(String status) {
+
+            }
+        };
+    }
+
+    @Override
+    public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        return new DeserializedOperatorNodePushable(ctx, new FileScanOperator(partition), plan, getActivityNodeId());
+    }
+
+    @Override
+    public boolean supportsPullInterface() {
+        return false;
+    }
+
+    @Override
+    public boolean supportsPushInterface() {
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/AbstractHadoopOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/AbstractHadoopOperatorDescriptor.java
new file mode 100644
index 0000000..4724d80
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/AbstractHadoopOperatorDescriptor.java
@@ -0,0 +1,172 @@
+/*
+ * 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.coreops.hadoop;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.Counters.Counter;
+
+import edu.uci.ics.dcache.client.DCacheClient;
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.coreops.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.hadoop.util.DatatypeHelper;
+import edu.uci.ics.hyracks.hadoop.util.IHadoopClassFactory;
+
+public abstract class AbstractHadoopOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+	
+	protected static class DataWritingOutputCollector<K, V> implements OutputCollector<K, V> {
+        private IDataWriter<Object[]> writer;
+
+        public DataWritingOutputCollector() {
+        }
+
+        public DataWritingOutputCollector(IDataWriter<Object[]> writer) {
+            this.writer = writer;
+        }
+
+        @Override
+        public void collect(Object key, Object value) throws IOException {
+           	writer.writeData(new Object[] { key, value });
+        }
+
+        public void setWriter(IDataWriter<Object[]> writer) {
+            this.writer = writer;
+        }
+    }
+
+    public static String MAPRED_CACHE_FILES = "mapred.cache.files";
+    public static String MAPRED_CACHE_LOCALFILES = "mapred.cache.localFiles";
+
+    private static final long serialVersionUID = 1L;
+    private final HashMap<String, String> jobConfMap;
+    private IHadoopClassFactory hadoopClassFactory;
+    
+    public abstract RecordDescriptor getRecordDescriptor(JobConf jobConf);
+    	
+    public AbstractHadoopOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor, JobConf jobConf,
+            IHadoopClassFactory hadoopOperatorFactory) {
+        super(spec, 1, 1);
+        jobConfMap = DatatypeHelper.jobConf2HashMap(jobConf);
+        this.hadoopClassFactory = hadoopOperatorFactory;
+        if(recordDescriptor != null){
+        	recordDescriptors[0]= recordDescriptor;
+        }else{
+        	recordDescriptors[0] = getRecordDescriptor(jobConf);
+        }
+    }
+
+     
+    public HashMap<String, String> getJobConfMap() {
+		return jobConfMap;
+	}
+
+
+	public IHadoopClassFactory getHadoopClassFactory() {
+		return hadoopClassFactory;
+	}
+
+
+	public void setHadoopClassFactory(IHadoopClassFactory hadoopClassFactory) {
+		this.hadoopClassFactory = hadoopClassFactory;
+	}
+
+
+	protected Reporter createReporter() {
+        return new Reporter() {
+            @Override
+            public Counter getCounter(Enum<?> name) {
+                return null;
+            }
+
+            @Override
+            public Counter getCounter(String group, String name) {
+                return null;
+            }    
+              
+
+            @Override
+            public InputSplit getInputSplit() throws UnsupportedOperationException {
+                return null;
+            }
+
+            @Override
+            public void incrCounter(Enum<?> key, long amount) {
+
+            }
+
+            @Override
+            public void incrCounter(String group, String counter, long amount) {
+
+            }
+
+            @Override
+            public void progress() {
+
+            }
+
+            @Override
+            public void setStatus(String status) {
+
+            }
+        };
+    }
+
+    public JobConf getJobConf() {
+        return DatatypeHelper.hashMap2JobConf(jobConfMap);
+    }
+    
+    public void populateCache(JobConf jobConf) {
+        String cache = jobConf.get(MAPRED_CACHE_FILES);
+        System.out.println("cache:" + cache);
+        if (cache == null) {
+            return;
+        }
+        String localCache = jobConf.get(MAPRED_CACHE_LOCALFILES);
+        System.out.println("localCache:" + localCache);
+        if (localCache != null) {
+            return;
+        }
+        localCache = "";
+        StringTokenizer cacheTokenizer = new StringTokenizer(cache, ",");
+        while (cacheTokenizer.hasMoreTokens()) {
+            if (!"".equals(localCache)) {
+                localCache += ",";
+            }
+            try {
+                localCache += DCacheClient.get().get(cacheTokenizer.nextToken());
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+        jobConf.set(MAPRED_CACHE_LOCALFILES, localCache);
+        System.out.println("localCache:" + localCache);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/HDFSWriteOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/HDFSWriteOperatorDescriptor.java
new file mode 100644
index 0000000..f2487f9
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/HDFSWriteOperatorDescriptor.java
@@ -0,0 +1,176 @@
+/*
+ * 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.coreops.hadoop;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.coreops.file.AbstractFileWriteOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.file.FileSplit;
+import edu.uci.ics.hyracks.coreops.file.IRecordWriter;
+import edu.uci.ics.hyracks.coreops.file.RecordWriter;
+import edu.uci.ics.hyracks.hadoop.util.DatatypeHelper;
+
+
+
+public class HDFSWriteOperatorDescriptor extends
+		AbstractFileWriteOperatorDescriptor {
+	
+	private static String nullWritableClassName = NullWritable.class.getName();
+	
+	private static class HDFSLineWriterImpl extends RecordWriter {
+		
+        HDFSLineWriterImpl(FileSystem fileSystem, String hdfsPath, int[] columns, char separator)
+                throws Exception {
+    		super(columns,separator,new Object[]{fileSystem,hdfsPath});
+        }
+
+		@Override
+		public OutputStream createOutputStream(Object[] args) throws Exception {
+			FSDataOutputStream fs = ((FileSystem)args[0]).create(new Path((String)args[1]));
+			return fs;
+		}
+
+		 @Override
+	     public void write(Object[] record) throws Exception {
+	         if(!nullWritableClassName.equals((record[0].getClass().getName()))){
+	             bufferedWriter.write(String.valueOf(record[0]));
+	         }
+	         if(!nullWritableClassName.equals((record[1].getClass().getName()))){
+	        	  bufferedWriter.write(separator);	 
+	        	  bufferedWriter.write(String.valueOf(record[1]));
+	         }	 
+	         bufferedWriter.write("\n");
+	     }
+    }
+
+	private static class HDFSSequenceWriterImpl extends RecordWriter {
+		
+		private Writer writer;
+		
+		HDFSSequenceWriterImpl(FileSystem fileSystem, String hdfsPath, Writer writer)
+                throws Exception {
+    		super(null,COMMA,new Object[]{fileSystem,hdfsPath});
+    		this.writer = writer;
+        }
+
+		@Override
+		public OutputStream createOutputStream(Object[] args) throws Exception {
+			return null;
+		}
+		
+		@Override
+	     public void close() {
+	         try {
+	             writer.close();
+	         } catch (IOException e) {
+	             e.printStackTrace();
+	         }
+	     }
+
+	     @Override
+	     public void write(Object[] record) throws Exception {
+	         Object key = record[0];
+	         Object value = record[1];
+	         writer.append(key, value);
+	     }
+
+    }
+	
+    private static final long serialVersionUID = 1L;
+    private static final char COMMA = ',';
+	private char separator;
+	private boolean sequenceFileOutput = false;
+	private String keyClassName;
+	private String valueClassName;
+	Map<String,String> jobConfMap;
+    
+
+    @Override
+    protected IRecordWriter createRecordWriter(File file,int index) throws Exception {
+    	JobConf conf = DatatypeHelper.hashMap2JobConf((HashMap)jobConfMap);
+		System.out.println("replication:" + conf.get("dfs.replication"));
+    	FileSystem fileSystem = null;
+		try{
+			fileSystem = FileSystem.get(conf);
+		}catch(IOException ioe){
+			ioe.printStackTrace();
+		}
+		Path path = new Path(file.getAbsolutePath());
+		checkIfCanWriteToHDFS(new FileSplit[]{new FileSplit("localhost",file)});
+		FSDataOutputStream outputStream = fileSystem.create(path);
+		outputStream.close();
+		if(sequenceFileOutput){
+			Class  keyClass = Class.forName(keyClassName);  
+			Class valueClass= Class.forName(valueClassName);
+			conf.setClass("mapred.output.compression.codec", GzipCodec.class, CompressionCodec.class);
+			Writer writer = SequenceFile.createWriter(fileSystem, conf,path, keyClass, valueClass);
+			return new HDFSSequenceWriterImpl(fileSystem, file.getAbsolutePath(), writer);
+		}else{
+			return new HDFSLineWriterImpl(fileSystem, file.getAbsolutePath(), null, COMMA);
+		}	
+    }
+    
+    private boolean checkIfCanWriteToHDFS(FileSplit[] fileSplits) throws Exception{
+    	boolean canWrite = true;
+    	JobConf conf = DatatypeHelper.hashMap2JobConf((HashMap)jobConfMap);
+		FileSystem fileSystem = null;
+		try{
+			fileSystem = FileSystem.get(conf);
+		    for(FileSplit fileSplit : fileSplits){
+				Path path = new Path(fileSplit.getLocalFile().getAbsolutePath());
+				canWrite = !fileSystem.exists(path);
+				if(!canWrite){
+					throw new Exception(" Output path :  already exists");
+				}	
+			}
+		}catch(IOException ioe){
+			ioe.printStackTrace();
+			throw ioe;
+		}
+	    return canWrite;
+    }
+		
+	
+	public HDFSWriteOperatorDescriptor(JobSpecification jobSpec,Map<String,String> jobConfMap, FileSplit[] fileSplits,char seperator,boolean sequenceFileOutput,String keyClassName, String valueClassName) throws Exception{
+		super(jobSpec,fileSplits);
+		this.jobConfMap = jobConfMap;
+		checkIfCanWriteToHDFS(fileSplits);
+		this.separator = seperator;
+		this.sequenceFileOutput = sequenceFileOutput;
+		this.keyClassName = keyClassName;
+		this.valueClassName = valueClassName;
+	}
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/HadoopMapperOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/HadoopMapperOperatorDescriptor.java
new file mode 100644
index 0000000..5385e44
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/HadoopMapperOperatorDescriptor.java
@@ -0,0 +1,157 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.hadoop;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.coreops.util.DeserializedOperatorNodePushable;
+import edu.uci.ics.hyracks.hadoop.util.ClasspathBasedHadoopClassFactory;
+import edu.uci.ics.hyracks.hadoop.util.DatatypeHelper;
+import edu.uci.ics.hyracks.hadoop.util.IHadoopClassFactory;
+
+public class HadoopMapperOperatorDescriptor<K1, V1, K2, V2> extends AbstractHadoopOperatorDescriptor {
+    private class MapperOperator implements IOpenableDataWriterOperator {
+        private OutputCollector<K2, V2> output;
+        private Reporter reporter;
+        private Mapper<K1, V1, K2, V2> mapper;
+        private IOpenableDataWriter<Object[]> writer;
+
+        @Override
+        public void close() throws HyracksDataException {
+            try {
+                mapper.close();
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+            writer.close();
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            JobConf jobConf = getJobConf();
+            populateCache(jobConf);
+            try {
+                mapper = createMapper();
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+            // -- - configure - --
+            mapper.configure(jobConf);
+            writer.open();
+            output = new DataWritingOutputCollector<K2, V2>(writer);
+            reporter = createReporter();
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            if (index != 0) {
+                throw new IllegalArgumentException();
+            }
+            this.writer = writer;
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            try {
+                mapper.map((K1) data[0], (V1) data[1], output, reporter);
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+    private static final String mapClassNameKey = "mapred.mapper.class";
+    private Class<? extends Mapper> mapperClass;
+   
+    public HadoopMapperOperatorDescriptor(JobSpecification spec, Class<? extends Mapper> mapperClass,
+            RecordDescriptor recordDescriptor, JobConf jobConf) {
+        super(spec, recordDescriptor, jobConf, null);
+        this.mapperClass = mapperClass;
+    }
+
+    public HadoopMapperOperatorDescriptor(JobSpecification spec, JobConf jobConf, IHadoopClassFactory hadoopClassFactory) {
+        super(spec, null, jobConf, hadoopClassFactory);
+    }
+
+    public RecordDescriptor getRecordDescriptor(JobConf conf){
+    	RecordDescriptor recordDescriptor = null;
+    	String mapOutputKeyClassName = conf.getMapOutputKeyClass().getName();
+		String mapOutputValueClassName = conf.getMapOutputValueClass().getName();
+		try{
+			if(getHadoopClassFactory() == null){
+				recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor((Class < ? extends Writable>) Class.forName(mapOutputKeyClassName),(Class < ? extends Writable>) Class.forName(mapOutputValueClassName));
+			}else{
+				recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor((Class < ? extends Writable>) getHadoopClassFactory().loadClass(mapOutputKeyClassName),(Class < ? extends Writable>) getHadoopClassFactory().loadClass(mapOutputValueClassName));
+			}
+		}catch(Exception e){
+			e.printStackTrace();
+		}
+		return recordDescriptor;
+    }
+
+    private Mapper<K1, V1, K2, V2> createMapper() throws Exception {
+        if (mapperClass != null) {
+            return mapperClass.newInstance();
+        } else {
+            String mapperClassName = super.getJobConfMap().get(mapClassNameKey);
+        	Object mapper = getHadoopClassFactory().createMapper(mapperClassName);
+            mapperClass = (Class<? extends Mapper>) mapper.getClass();
+            return (Mapper)mapper;
+        }
+    }
+
+    @Override
+    public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        return new DeserializedOperatorNodePushable(ctx, new MapperOperator(), plan, getActivityNodeId());
+    }
+
+    public Class<? extends Mapper> getMapperClass() {
+        return mapperClass;
+    }
+
+    @Override
+    public boolean supportsPullInterface() {
+        return false;
+    }
+
+    @Override
+    public boolean supportsPushInterface() {
+        return true;
+    }
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/HadoopReadOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/HadoopReadOperatorDescriptor.java
new file mode 100644
index 0000000..c1b917d
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/HadoopReadOperatorDescriptor.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.coreops.hadoop;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.NetworkInterface;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.SequenceFileRecordReader;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.controller.IClusterController;
+import edu.uci.ics.hyracks.api.controller.INodeController;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.config.NCConfig;
+import edu.uci.ics.hyracks.coreops.file.IRecordReader;
+import edu.uci.ics.hyracks.hadoop.util.DatatypeHelper;
+import edu.uci.ics.hyracks.hadoop.util.HadoopAdapter;
+import edu.uci.ics.hyracks.hadoop.util.HadoopFileSplit;
+
+public class HadoopReadOperatorDescriptor extends AbstractHadoopFileScanOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private String inputFormatClassName;
+    private Map<String, String> jobConfMap;
+
+    private static class HDFSCustomReader implements IRecordReader {
+        private RecordReader hadoopRecordReader;
+        private Class inputKeyClass;
+        private Class inputValueClass;
+        private Object key;
+        private Object value;
+
+        public HDFSCustomReader(Map<String, String> jobConfMap, HadoopFileSplit inputSplit,
+                String inputFormatClassName, Reporter reporter) {
+            try {
+                JobConf conf = DatatypeHelper.hashMap2JobConf((HashMap) jobConfMap);
+                FileSystem fileSystem = null;
+                try {
+                    fileSystem = FileSystem.get(conf);
+                } catch (IOException ioe) {
+                    ioe.printStackTrace();
+                }
+
+                Class inputFormatClass = Class.forName(inputFormatClassName);
+                InputFormat inputFormat = (InputFormat) ReflectionUtils.newInstance(inputFormatClass, conf);
+                hadoopRecordReader = (RecordReader) inputFormat.getRecordReader(getFileSplit(inputSplit), conf,
+                        reporter);
+                if (hadoopRecordReader instanceof SequenceFileRecordReader) {
+                    inputKeyClass = ((SequenceFileRecordReader) hadoopRecordReader).getKeyClass();
+                    inputValueClass = ((SequenceFileRecordReader) hadoopRecordReader).getValueClass();
+                } else {
+                    inputKeyClass = hadoopRecordReader.createKey().getClass();
+                    inputValueClass = hadoopRecordReader.createValue().getClass();
+                }
+                key = inputKeyClass.newInstance();
+                value = inputValueClass.newInstance();
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+
+        public Class getInputKeyClass() {
+            return inputKeyClass;
+        }
+
+        public void setInputKeyClass(Class inputKeyClass) {
+            this.inputKeyClass = inputKeyClass;
+        }
+
+        public Class getInputValueClass() {
+            return inputValueClass;
+        }
+
+        public void setInputValueClass(Class inputValueClass) {
+            this.inputValueClass = inputValueClass;
+        }
+
+        @Override
+        public void close() {
+            try {
+                hadoopRecordReader.close();
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+
+        @Override
+        public boolean read(Object[] record) throws Exception {
+            if (!hadoopRecordReader.next(key, value)) {
+                return false;
+            }
+            if (record.length == 1) {
+                record[0] = value;
+            } else {
+                record[0] = key;
+                record[1] = value;
+            }
+            return true;
+        }
+
+        private FileSplit getFileSplit(HadoopFileSplit hadoopFileSplit) {
+            FileSplit fileSplit = new FileSplit(new Path(hadoopFileSplit.getFile()), hadoopFileSplit.getStart(),
+                    hadoopFileSplit.getLength(), hadoopFileSplit.getHosts());
+            return fileSplit;
+        }
+    }
+
+    public HadoopReadOperatorDescriptor(Map<String, String> jobConfMap, JobSpecification spec,
+            HadoopFileSplit[] splits, String inputFormatClassName, RecordDescriptor recordDescriptor) {
+        super(spec, splits, recordDescriptor);
+        this.inputFormatClassName = inputFormatClassName;
+        this.jobConfMap = jobConfMap;
+    }
+
+    public HadoopReadOperatorDescriptor(Map<String, String> jobConfMap, InetSocketAddress nameNode,
+            JobSpecification spec, String inputFormatClassName, RecordDescriptor recordDescriptor) {
+        super(spec, null, recordDescriptor);
+        this.inputFormatClassName = inputFormatClassName;
+        this.jobConfMap = jobConfMap;
+    }
+
+    public HadoopReadOperatorDescriptor(IClusterController clusterController, Map<String, String> jobConfMap,
+            JobSpecification spec, String fileSystemURL, String inputFormatClassName, RecordDescriptor recordDescriptor) {
+        super(spec, null, recordDescriptor);
+        HadoopAdapter hadoopAdapter = HadoopAdapter.getInstance(fileSystemURL);
+        String inputPathString = jobConfMap.get("mapred.input.dir");
+        String[] inputPaths = inputPathString.split(",");
+        Map<String, List<HadoopFileSplit>> blocksToRead = hadoopAdapter.getInputSplits(inputPaths);
+        List<HadoopFileSplit> hadoopFileSplits = new ArrayList<HadoopFileSplit>();
+        for (String filePath : blocksToRead.keySet()) {
+            hadoopFileSplits.addAll(blocksToRead.get(filePath));
+        }
+        for (HadoopFileSplit hadoopFileSplit : hadoopFileSplits) {
+            System.out.println(" Hadoop File Split : " + hadoopFileSplit);
+        }
+        super.splits = hadoopFileSplits.toArray(new HadoopFileSplit[] {});
+        configurePartitionConstraints(clusterController, blocksToRead);
+        this.inputFormatClassName = inputFormatClassName;
+        this.jobConfMap = jobConfMap;
+    }
+
+    private void configurePartitionConstraints(IClusterController clusterController,
+            Map<String, List<HadoopFileSplit>> blocksToRead) {
+        List<LocationConstraint> locationConstraints = new ArrayList<LocationConstraint>();
+        Map<String, INodeController> registry = null;
+        try {
+            registry = clusterController.getRegistry();
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+        Map<String, String> hostnameToNodeIdMap = new HashMap<String, String>();
+        NCConfig ncConfig = null;
+        for (String nodeId : registry.keySet()) {
+            try {
+                ncConfig = ((INodeController) registry.get(nodeId)).getConfiguration();
+                String ipAddress = ncConfig.dataIPAddress;
+                String hostname = InetAddress.getByName(ipAddress).getHostName();
+                System.out.println(" hostname :" + hostname + " nodeid:" + nodeId);
+                hostnameToNodeIdMap.put(hostname, nodeId);
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+
+        for (String filePath : blocksToRead.keySet()) {
+            List<HadoopFileSplit> hadoopFileSplits = blocksToRead.get(filePath);
+            for (HadoopFileSplit hadoopFileSplit : hadoopFileSplits) {
+                String hostname = hadoopFileSplit.getHosts()[0];
+                System.out.println("host name is :" + hostname);
+                InetAddress address = null;
+                try {
+                    address = InetAddress.getByName(hostname);
+                    if (address.isLoopbackAddress()) {
+                        Enumeration<NetworkInterface> netInterfaces = NetworkInterface.getNetworkInterfaces();
+                        while (netInterfaces.hasMoreElements()) {
+                            NetworkInterface ni = netInterfaces.nextElement();
+                            InetAddress inetAddress = (InetAddress) ni.getInetAddresses().nextElement();
+                            if (!inetAddress.isLoopbackAddress()) {
+                                address = inetAddress;
+                                break;
+                            }
+                        }
+                    }
+                    hostname = address.getHostName();
+                    System.out.println("cannonical host name hyracks :" + hostname);
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+                String nodeId = hostnameToNodeIdMap.get(hostname);
+                System.out.println(" corresponding node id is :" + nodeId);
+                LocationConstraint locationConstraint = new AbsoluteLocationConstraint(nodeId);
+                locationConstraints.add(locationConstraint);
+            }
+        }
+
+        PartitionConstraint partitionConstraint = new PartitionConstraint(locationConstraints
+                .toArray(new LocationConstraint[] {}));
+        this.setPartitionConstraint(partitionConstraint);
+    }
+
+    @Override
+    protected IRecordReader createRecordReader(HadoopFileSplit fileSplit, RecordDescriptor desc) throws Exception {
+        Reporter reporter = createReporter();
+        IRecordReader recordReader = new HDFSCustomReader(jobConfMap, fileSplit, inputFormatClassName, reporter);
+        return recordReader;
+    }
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/HadoopReducerOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/HadoopReducerOperatorDescriptor.java
new file mode 100644
index 0000000..7881d45
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/HadoopReducerOperatorDescriptor.java
@@ -0,0 +1,280 @@
+/*
+ * 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.coreops.hadoop;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.Counters.Counter;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.coreops.group.IGroupAggregator;
+import edu.uci.ics.hyracks.coreops.group.PreclusteredGroupOperator;
+import edu.uci.ics.hyracks.coreops.hadoop.data.KeyComparatorFactory;
+import edu.uci.ics.hyracks.coreops.hadoop.data.WritableComparingComparatorFactory;
+import edu.uci.ics.hyracks.coreops.util.DeserializedOperatorNodePushable;
+import edu.uci.ics.hyracks.hadoop.util.ClasspathBasedHadoopClassFactory;
+import edu.uci.ics.hyracks.hadoop.util.DatatypeHelper;
+import edu.uci.ics.hyracks.hadoop.util.IHadoopClassFactory;
+
+public class HadoopReducerOperatorDescriptor<K2, V2, K3, V3> extends AbstractHadoopOperatorDescriptor {
+    private class ReducerAggregator implements IGroupAggregator {
+        private Reducer<K2, V2, K3, V3> reducer;
+        private DataWritingOutputCollector<K3, V3> output;
+        private Reporter reporter;
+
+        public ReducerAggregator(Reducer<K2, V2, K3, V3> reducer) {
+            this.reducer = reducer;
+            reducer.configure(getJobConf());
+            output = new DataWritingOutputCollector<K3, V3>();
+            reporter = new Reporter() {
+
+                @Override
+                public void progress() {
+
+                }
+
+                @Override
+                public void setStatus(String arg0) {
+
+                }
+
+                @Override
+                public void incrCounter(String arg0, String arg1, long arg2) {
+
+                }
+
+                @Override
+                public void incrCounter(Enum<?> arg0, long arg1) {
+
+                }
+
+                @Override
+                public InputSplit getInputSplit() throws UnsupportedOperationException {
+                    return null;
+                }
+
+                @Override
+                public Counter getCounter(String arg0, String arg1) {
+                    return null;
+                }
+
+                @Override
+                public Counter getCounter(Enum<?> arg0) {
+                    return null;
+                }
+            };
+        }
+
+        @Override
+        public void aggregate(IDataReader<Object[]> reader, IDataWriter<Object[]> writer) throws HyracksDataException {
+
+            ValueIterator i = new ValueIterator();
+            i.reset(reader);
+            output.setWriter(writer);
+            try {
+
+                // -- - reduce - --
+                reducer.reduce(i.getKey(), i, output, reporter);
+
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            // -- - close - --
+            try {
+                reducer.close();
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    private class ValueIterator implements Iterator<V2> {
+        private IDataReader<Object[]> reader;
+        private K2 key;
+        private V2 value;
+
+        public K2 getKey() {
+            return key;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (value == null) {
+                Object[] tuple;
+                try {
+                    tuple = reader.readData();
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+                if (tuple != null) {
+                    value = (V2) tuple[1];
+                }
+            }
+            return value != null;
+        }
+
+        @Override
+        public V2 next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            V2 v = value;
+            value = null;
+            return v;
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException();
+        }
+
+        void reset(IDataReader<Object[]> reader) {
+            this.reader = reader;
+            try {
+                Object[] tuple = reader.readData();
+                key = (K2) tuple[0];
+                value = (V2) tuple[1];
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+    private Class<? extends Reducer> reducerClass;
+    private static final String reducerClassKey = "mapred.reducer.class";
+    private static final String comparatorClassKey = "mapred.output.value.groupfn.class";
+    private IComparatorFactory comparatorFactory;
+    
+
+    public HadoopReducerOperatorDescriptor(JobSpecification spec, IComparatorFactory comparatorFactory,
+            Class<? extends Reducer> reducerClass, RecordDescriptor recordDescriptor, JobConf jobConf) {
+        super(spec, recordDescriptor, jobConf, new ClasspathBasedHadoopClassFactory());
+        this.comparatorFactory = comparatorFactory;
+        this.reducerClass = reducerClass;
+    }
+
+    
+    public HadoopReducerOperatorDescriptor(JobSpecification spec, JobConf conf, IHadoopClassFactory classFactory) {
+        super(spec, null , conf, classFactory);
+    }
+    
+    private Reducer<K2, V2, K3, V3> createReducer() throws Exception {
+        if(reducerClass != null){
+        	return reducerClass.newInstance();
+        }else{
+        	Object reducer = getHadoopClassFactory().createReducer(getJobConfMap().get(reducerClassKey));
+            reducerClass = (Class<? extends Reducer>) reducer.getClass();
+            return (Reducer)reducer;	
+        }
+    }
+
+    @Override
+    public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+            int partition) {
+        try {
+        	if(this.comparatorFactory == null){
+	            String comparatorClassName  = getJobConfMap().get(comparatorClassKey);
+	            RawComparator rawComparator = null;
+	            if(comparatorClassName != null){
+		            Class comparatorClazz = getHadoopClassFactory().loadClass(comparatorClassName);
+				    this.comparatorFactory = new KeyComparatorFactory(comparatorClazz);
+		            
+			    }else{
+		        	String mapOutputKeyClass = getJobConfMap().get("mapred.mapoutput.key.class");
+		        	if(getHadoopClassFactory() != null){
+		        		rawComparator = WritableComparator.get(getHadoopClassFactory().loadClass(mapOutputKeyClass));
+		        	}else{
+		        		rawComparator = WritableComparator.get((Class<? extends WritableComparable>)Class.forName(mapOutputKeyClass));
+		        	}
+		        	this.comparatorFactory = new WritableComparingComparatorFactory(rawComparator.getClass());
+		        }
+        	}
+	        IOpenableDataWriterOperator op = new PreclusteredGroupOperator(new int[] { 0 },
+                    new IComparator[] { comparatorFactory.createComparator() }, new ReducerAggregator(createReducer()));
+            return new DeserializedOperatorNodePushable(ctx, op, plan, getActivityNodeId());
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public Class<? extends Reducer> getReducerClass() {
+        return reducerClass;
+    }
+
+    public void setReducerClass(Class<? extends Reducer> reducerClass) {
+        this.reducerClass = reducerClass;
+    }
+
+    @Override
+    public boolean supportsPullInterface() {
+        return false;
+    }
+
+    @Override
+    public boolean supportsPushInterface() {
+        return true;
+    }
+
+	@Override
+	public RecordDescriptor getRecordDescriptor(JobConf conf) {
+		String outputKeyClassName = conf.get("mapred.output.key.class");
+		String outputValueClassName = conf.get("mapred.output.value.class");
+		RecordDescriptor recordDescriptor = null;
+		try{
+			if(getHadoopClassFactory() == null){
+				recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor((Class<? extends Writable>)Class.forName(outputKeyClassName), (Class<? extends Writable>)Class.forName(outputValueClassName));
+			}else{
+				recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor((Class<? extends Writable>)getHadoopClassFactory().loadClass(outputKeyClassName), (Class<? extends Writable>)getHadoopClassFactory().loadClass(outputValueClassName));
+			}
+		}catch(Exception e){
+			e.printStackTrace();
+			return null;
+		}	
+		return recordDescriptor;
+	}
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/AbstractClassBasedDelegate.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/AbstractClassBasedDelegate.java
new file mode 100644
index 0000000..afc8ccb
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/AbstractClassBasedDelegate.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.coreops.hadoop.data;
+
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+
+public class AbstractClassBasedDelegate<T> implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private Class<? extends T> klass;
+    protected transient T instance;
+
+    public AbstractClassBasedDelegate(Class<? extends T> klass) {
+        this.klass = klass;
+        init();
+    }
+
+    protected Object readResolve() throws ObjectStreamException {
+        init();
+        return this;
+    }
+
+    private void init() {
+        try {
+            instance = klass.newInstance();
+        } catch (InstantiationException e) {
+            throw new RuntimeException(e);
+        } catch (IllegalAccessException e) {
+            throw new RuntimeException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/HadoopHashTuplePartitionComputerFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
new file mode 100644
index 0000000..e361ebb
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/HadoopHashTuplePartitionComputerFactory.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.coreops.hadoop.data;
+
+import java.io.DataInputStream;
+
+import org.apache.hadoop.io.Writable;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.util.ByteBufferInputStream;
+
+public class HadoopHashTuplePartitionComputerFactory<K extends Writable> implements ITuplePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+    private final ISerializerDeserializer<K> keyIO;
+
+    public HadoopHashTuplePartitionComputerFactory(ISerializerDeserializer<K> keyIO) {
+        this.keyIO = keyIO;
+    }
+
+    @Override
+    public ITuplePartitionComputer createPartitioner() {
+        return new ITuplePartitionComputer() {
+            private final ByteBufferInputStream bbis = new ByteBufferInputStream();
+            private final DataInputStream dis = new DataInputStream(bbis);
+
+            @Override
+            public int partition(FrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+                int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
+                        + accessor.getFieldStartOffset(tIndex, 0);
+                bbis.setByteBuffer(accessor.getBuffer(), keyStart);
+                K key = keyIO.deserialize(dis);
+                int h = key.hashCode();
+                if (h < 0) {
+                    h = -h;
+                }
+                return h % nParts;
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
new file mode 100644
index 0000000..a80682f
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.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.coreops.hadoop.data;
+
+import java.io.DataInputStream;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Partitioner;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.util.ByteBufferInputStream;
+
+public class HadoopPartitionerTuplePartitionComputerFactory<K extends Writable, V extends Writable> extends
+        AbstractClassBasedDelegate<Partitioner<K, V>> implements ITuplePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+    private final ISerializerDeserializer<K> keyIO;
+    private final ISerializerDeserializer<V> valueIO;
+
+    public HadoopPartitionerTuplePartitionComputerFactory(Class<? extends Partitioner<K, V>> klass,
+            ISerializerDeserializer<K> keyIO, ISerializerDeserializer<V> valueIO) {
+        super(klass);
+        this.keyIO = keyIO;
+        this.valueIO = valueIO;
+    }
+
+    @Override
+    public ITuplePartitionComputer createPartitioner() {
+        return new ITuplePartitionComputer() {
+            private final ByteBufferInputStream bbis = new ByteBufferInputStream();
+            private final DataInputStream dis = new DataInputStream(bbis);
+
+            @Override
+            public int partition(FrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+                int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
+                        + accessor.getFieldStartOffset(tIndex, 0);
+                bbis.setByteBuffer(accessor.getBuffer(), keyStart);
+                K key = keyIO.deserialize(dis);
+                int valueStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
+                        + accessor.getFieldStartOffset(tIndex, 1);
+                bbis.setByteBuffer(accessor.getBuffer(), valueStart);
+                V value = valueIO.deserialize(dis);
+                return instance.getPartition(key, value, nParts);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/KeyBinaryComparatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/KeyBinaryComparatorFactory.java
new file mode 100644
index 0000000..8b16fc4
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/KeyBinaryComparatorFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.coreops.hadoop.data;
+
+import org.apache.hadoop.io.RawComparator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.util.ReflectionUtils;
+
+public class KeyBinaryComparatorFactory<T> implements IBinaryComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    private Class<? extends RawComparator<T>> cmpClass;
+
+    public KeyBinaryComparatorFactory(Class<? extends RawComparator<T>> cmpClass) {
+        this.cmpClass = cmpClass;
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        final RawComparator<T> instance = ReflectionUtils.createInstance(cmpClass);
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                return instance.compare(b1, s1, l1, b2, s2, l2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/KeyComparatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/KeyComparatorFactory.java
new file mode 100644
index 0000000..9f48251
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/KeyComparatorFactory.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.coreops.hadoop.data;
+
+import org.apache.hadoop.io.RawComparator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+import edu.uci.ics.hyracks.util.ReflectionUtils;
+
+public class KeyComparatorFactory<T> implements IComparatorFactory<T> {
+    private static final long serialVersionUID = 1L;
+    private Class<? extends RawComparator<T>> cmpClass;
+
+    public KeyComparatorFactory(Class<? extends RawComparator<T>> cmpClass) {
+        this.cmpClass = cmpClass;
+    }
+
+    @Override
+    public IComparator<T> createComparator() {
+        final RawComparator<T> instance = ReflectionUtils.createInstance(cmpClass);
+        return new IComparator<T>() {
+            @Override
+            public int compare(T o1, T o2) {
+                return instance.compare(o1, o2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/WritableComparingBinaryComparatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/WritableComparingBinaryComparatorFactory.java
new file mode 100644
index 0000000..b633742
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/WritableComparingBinaryComparatorFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.coreops.hadoop.data;
+
+import org.apache.hadoop.io.WritableComparator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.util.ReflectionUtils;
+
+public class WritableComparingBinaryComparatorFactory<T> implements IBinaryComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    private Class<? extends WritableComparator> cmpClass;
+
+    public WritableComparingBinaryComparatorFactory(Class<? extends WritableComparator> cmpClass) {
+        this.cmpClass = cmpClass;
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        final WritableComparator instance = ReflectionUtils.createInstance(cmpClass);
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                return instance.compare(b1, s1, l1, b2, s2, l2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/WritableComparingComparatorFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/WritableComparingComparatorFactory.java
new file mode 100644
index 0000000..c53abcc
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/hadoop/data/WritableComparingComparatorFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.hadoop.data;
+
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+import edu.uci.ics.hyracks.util.ReflectionUtils;
+
+public class WritableComparingComparatorFactory<T> implements IComparatorFactory<WritableComparable<T>> {
+    private Class<? extends WritableComparator> klass;
+
+    public WritableComparingComparatorFactory(Class<? extends WritableComparator> klass) {
+        this.klass = klass;
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IComparator<WritableComparable<T>> createComparator() {
+        final WritableComparator instance = ReflectionUtils.createInstance(klass);
+        return new IComparator<WritableComparable<T>>() {
+            @Override
+            public int compare(WritableComparable<T> o1, WritableComparable<T> o2) {
+                return instance.compare(o1, o2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/join/InMemoryHashJoin.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/join/InMemoryHashJoin.java
new file mode 100644
index 0000000..0962895
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/join/InMemoryHashJoin.java
@@ -0,0 +1,131 @@
+/*
+ * 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.coreops.join;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.comm.io.FrameTuplePairComparator;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class InMemoryHashJoin {
+    private final Link[] table;
+    private final List<ByteBuffer> buffers;
+    private final FrameTupleAccessor accessor0;
+    private final ITuplePartitionComputer tpc0;
+    private final FrameTupleAccessor accessor1;
+    private final ITuplePartitionComputer tpc1;
+    private final FrameTupleAppender appender;
+    private final FrameTuplePairComparator tpComparator;
+    private final ByteBuffer outBuffer;
+
+    public InMemoryHashJoin(HyracksContext ctx, int tableSize, FrameTupleAccessor accessor0,
+            ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
+            FrameTuplePairComparator comparator) {
+        table = new Link[tableSize];
+        buffers = new ArrayList<ByteBuffer>();
+        this.accessor0 = accessor0;
+        this.tpc0 = tpc0;
+        this.accessor1 = accessor1;
+        this.tpc1 = tpc1;
+        appender = new FrameTupleAppender(ctx);
+        tpComparator = comparator;
+        outBuffer = ctx.getResourceManager().allocateFrame();
+        appender.reset(outBuffer, true);
+    }
+
+    public void build(ByteBuffer buffer) throws HyracksDataException {
+        buffers.add(buffer);
+        int bIndex = buffers.size() - 1;
+        accessor0.reset(buffer);
+        int tCount = accessor0.getTupleCount();
+        for (int i = 0; i < tCount; ++i) {
+            int entry = tpc0.partition(accessor0, i, table.length);
+            long tPointer = (((long) bIndex) << 32) + i;
+            Link link = table[entry];
+            if (link == null) {
+                link = table[entry] = new Link();
+            }
+            link.add(tPointer);
+        }
+    }
+
+    public void join(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
+        accessor1.reset(buffer);
+        int tupleCount1 = accessor1.getTupleCount();
+        for (int i = 0; i < tupleCount1; ++i) {
+            int entry = tpc1.partition(accessor1, i, table.length);
+            Link link = table[entry];
+            if (link != null) {
+                for (int j = 0; j < link.size; ++j) {
+                    long pointer = link.pointers[j];
+                    int bIndex = (int) ((pointer >> 32) & 0xffffffff);
+                    int tIndex = (int) (pointer & 0xffffffff);
+                    accessor0.reset(buffers.get(bIndex));
+                    int c = tpComparator.compare(accessor0, tIndex, accessor1, i);
+                    if (c == 0) {
+                        if (!appender.appendConcat(accessor0, tIndex, accessor1, i)) {
+                            flushFrame(outBuffer, writer);
+                            appender.reset(outBuffer, true);
+                            if (!appender.appendConcat(accessor0, tIndex, accessor1, i)) {
+                                throw new IllegalStateException();
+                            }
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    public void closeJoin(IFrameWriter writer) throws HyracksDataException {
+        if (appender.getTupleCount() > 0) {
+            flushFrame(outBuffer, writer);
+        }
+    }
+
+    private void flushFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
+        buffer.position(0);
+        buffer.limit(buffer.capacity());
+        writer.nextFrame(outBuffer);
+        buffer.position(0);
+        buffer.limit(buffer.capacity());
+    }
+
+    private static class Link {
+        private static final int INIT_POINTERS_SIZE = 8;
+
+        long[] pointers;
+        int size;
+
+        Link() {
+            pointers = new long[INIT_POINTERS_SIZE];
+            size = 0;
+        }
+
+        void add(long pointer) {
+            if (size >= pointers.length) {
+                pointers = Arrays.copyOf(pointers, pointers.length * 2);
+            }
+            pointers[size++] = pointer;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/join/InMemoryHashJoinOperatorDescriptor.java
new file mode 100644
index 0000000..38fbc69
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/join/InMemoryHashJoinOperatorDescriptor.java
@@ -0,0 +1,208 @@
+/*
+ * 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.coreops.join;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePullable;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.comm.io.FrameTuplePairComparator;
+import edu.uci.ics.hyracks.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.coreops.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.coreops.base.AbstractOperatorDescriptor;
+
+public class InMemoryHashJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final String JOINER = "joiner";
+
+    private static final long serialVersionUID = 1L;
+    private final int[] keys0;
+    private final int[] keys1;
+    private final IBinaryHashFunctionFactory[] hashFunctionFactories;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final int tableSize;
+
+    public InMemoryHashJoinOperatorDescriptor(JobSpecification spec, int[] keys0, int[] keys1,
+            IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, int tableSize) {
+        super(spec, 2, 1);
+        this.keys0 = keys0;
+        this.keys1 = keys1;
+        this.hashFunctionFactories = hashFunctionFactories;
+        this.comparatorFactories = comparatorFactories;
+        this.tableSize = tableSize;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeTaskGraph(IActivityGraphBuilder builder) {
+        HashBuildActivityNode hba = new HashBuildActivityNode();
+        HashProbeActivityNode hpa = new HashProbeActivityNode();
+
+        builder.addTask(hba);
+        builder.addSourceEdge(0, hba, 0);
+
+        builder.addTask(hpa);
+        builder.addSourceEdge(1, hpa, 0);
+        builder.addTargetEdge(0, hpa, 0);
+
+        builder.addBlockingEdge(hba, hpa);
+    }
+
+    private class HashBuildActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+                int partition) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final HyracksContext ctx, JobPlan plan,
+                final IOperatorEnvironment env, int partition) {
+            final RecordDescriptor rd0 = plan.getJobSpecification()
+                    .getOperatorInputRecordDescriptor(getOperatorId(), 0);
+            final RecordDescriptor rd1 = plan.getJobSpecification()
+                    .getOperatorInputRecordDescriptor(getOperatorId(), 1);
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private InMemoryHashJoin joiner;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    ITuplePartitionComputer hpc0 = new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories)
+                            .createPartitioner();
+                    ITuplePartitionComputer hpc1 = new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories)
+                            .createPartitioner();
+                    joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(ctx, rd0), hpc0,
+                            new FrameTupleAccessor(ctx, rd1), hpc1, new FrameTuplePairComparator(keys0, keys1,
+                                    comparators));
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    ByteBuffer copyBuffer = ctx.getResourceManager().allocateFrame();
+                    FrameUtils.copy(buffer, copyBuffer);
+                    joiner.build(copyBuffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    env.set(JOINER, joiner);
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer) {
+                    throw new IllegalArgumentException();
+                }
+            };
+            return op;
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return InMemoryHashJoinOperatorDescriptor.this;
+        }
+
+        @Override
+        public boolean supportsPullInterface() {
+            return false;
+        }
+
+        @Override
+        public boolean supportsPushInterface() {
+            return true;
+        }
+    }
+
+    private class HashProbeActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IOperatorNodePullable createPullRuntime(HyracksContext ctx, JobPlan plan, IOperatorEnvironment env,
+                int partition) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final HyracksContext ctx, JobPlan plan,
+                final IOperatorEnvironment env, int partition) {
+            IOperatorNodePushable op = new IOperatorNodePushable() {
+                private IFrameWriter writer;
+                private InMemoryHashJoin joiner;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    joiner = (InMemoryHashJoin) env.get(JOINER);
+                    writer.open();
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    joiner.join(buffer, writer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    joiner.closeJoin(writer);
+                    writer.close();
+                    env.set(JOINER, null);
+                }
+
+                @Override
+                public void setFrameWriter(int index, IFrameWriter writer) {
+                    if (index != 0) {
+                        throw new IllegalStateException();
+                    }
+                    this.writer = writer;
+                }
+            };
+            return op;
+        }
+
+        @Override
+        public IOperatorDescriptor getOwner() {
+            return InMemoryHashJoinOperatorDescriptor.this;
+        }
+
+        @Override
+        public boolean supportsPullInterface() {
+            return false;
+        }
+
+        @Override
+        public boolean supportsPushInterface() {
+            return true;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/map/ReflectionBasedMapperFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/map/ReflectionBasedMapperFactory.java
new file mode 100644
index 0000000..c284ff1
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/map/ReflectionBasedMapperFactory.java
@@ -0,0 +1,40 @@
+/*
+ * 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.coreops.map;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.coreops.IMapper;
+import edu.uci.ics.hyracks.coreops.IMapperFactory;
+
+public class ReflectionBasedMapperFactory implements IMapperFactory {
+    private static final long serialVersionUID = 1L;
+
+    private final Class<? extends IMapper> mapperClass;
+
+    public ReflectionBasedMapperFactory(Class<? extends IMapper> mapperClass) {
+        this.mapperClass = mapperClass;
+    }
+
+    @Override
+    public IMapper createMapper() throws HyracksDataException {
+        try {
+            return mapperClass.newInstance();
+        } catch (InstantiationException e) {
+            throw new HyracksDataException(e);
+        } catch (IllegalAccessException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/map/SamplerMapFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/map/SamplerMapFactory.java
new file mode 100644
index 0000000..44541a1
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/map/SamplerMapFactory.java
@@ -0,0 +1,45 @@
+/*
+ * 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.coreops.map;
+
+import java.util.Random;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.coreops.IMapper;
+import edu.uci.ics.hyracks.coreops.IMapperFactory;
+
+public class SamplerMapFactory implements IMapperFactory {
+    private static final long serialVersionUID = 1L;
+    private final float sampleRatio;
+
+    public SamplerMapFactory(float sampleRatio) {
+        this.sampleRatio = sampleRatio;
+    }
+
+    @Override
+    public IMapper createMapper() throws HyracksDataException {
+        return new IMapper() {
+            private Random random = new Random();
+
+            @Override
+            public void map(Object[] data, IDataWriter<Object[]> writer) throws HyracksDataException {
+                if (random.nextFloat() <= sampleRatio) {
+                    writer.writeData(data);
+                }
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/DeserializedOperatorNodePushable.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/DeserializedOperatorNodePushable.java
new file mode 100644
index 0000000..610cc51
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/DeserializedOperatorNodePushable.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.util;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.comm.io.FrameDeserializer;
+import edu.uci.ics.hyracks.comm.io.SerializingDataWriter;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.base.IOpenableDataWriterOperator;
+
+public final class DeserializedOperatorNodePushable implements IOperatorNodePushable {
+    private final HyracksContext ctx;
+
+    private final IOpenableDataWriterOperator delegate;
+
+    private final JobPlan plan;
+
+    private final ActivityNodeId aid;
+
+    private final FrameDeserializer deserializer;
+
+    public DeserializedOperatorNodePushable(HyracksContext ctx, IOpenableDataWriterOperator delegate, JobPlan plan,
+            ActivityNodeId aid) {
+        this.ctx = ctx;
+        this.delegate = delegate;
+        this.plan = plan;
+        this.aid = aid;
+        List<Integer> inList = plan.getTaskInputMap().get(aid);
+        deserializer = inList == null ? null : new FrameDeserializer(ctx, plan.getTaskInputRecordDescriptor(aid, 0));
+    }
+
+    @Override
+    public void setFrameWriter(int index, IFrameWriter writer) {
+        delegate.setDataWriter(index, new SerializingDataWriter(ctx, plan.getTaskOutputRecordDescriptor(aid, index),
+                writer));
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        delegate.close();
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        deserializer.reset(buffer);
+        while (!deserializer.done()) {
+            delegate.writeData(deserializer.deserializeRecord());
+        }
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        delegate.open();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/ReferenceEntry.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/ReferenceEntry.java
new file mode 100644
index 0000000..025747d
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/ReferenceEntry.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.util;
+
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+
+public class ReferenceEntry {
+    private final int runid;
+    private FrameTupleAccessor acccessor;
+    private int tupleIndex;
+
+    public ReferenceEntry(int runid, FrameTupleAccessor fta, int tupleIndex) {
+        super();
+        this.runid = runid;
+        this.acccessor = fta;
+        this.tupleIndex = tupleIndex;
+    }
+
+    public int getRunid() {
+        return runid;
+    }
+
+    public FrameTupleAccessor getAccessor() {
+        return acccessor;
+    }
+
+    public void setAccessor(FrameTupleAccessor fta) {
+        this.acccessor = fta;
+    }
+
+    public int getTupleIndex() {
+        return tupleIndex;
+    }
+
+    public void setTupleIndex(int tupleIndex) {
+        this.tupleIndex = tupleIndex;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/ReferencedPriorityQueue.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/ReferencedPriorityQueue.java
new file mode 100644
index 0000000..f64320b
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/ReferencedPriorityQueue.java
@@ -0,0 +1,144 @@
+/*
+ * 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.coreops.util;
+
+import java.io.IOException;
+import java.util.BitSet;
+import java.util.Comparator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class ReferencedPriorityQueue {
+    private final HyracksContext ctx;
+    private final RecordDescriptor recordDescriptor;
+    private final ReferenceEntry entries[];
+    private final int size;
+    private final BitSet runAvail;
+    private int nItems;
+
+    private final Comparator<ReferenceEntry> comparator;
+
+    public ReferencedPriorityQueue(HyracksContext ctx, RecordDescriptor recordDescriptor, int initSize,
+            Comparator<ReferenceEntry> comparator) throws IOException {
+        this.ctx = ctx;
+        this.recordDescriptor = recordDescriptor;
+        if (initSize < 1)
+            throw new IllegalArgumentException();
+        this.comparator = comparator;
+        nItems = initSize;
+        size = (initSize + 1) & 0xfffffffe;
+        entries = new ReferenceEntry[size];
+        runAvail = new BitSet(size);
+        runAvail.set(0, initSize, true);
+        for (int i = 0; i < size; i++) {
+            entries[i] = new ReferenceEntry(i, null, -1);
+        }
+    }
+
+    /**
+     * Retrieve the top entry without removing it
+     * 
+     * @return the top entry
+     */
+    public ReferenceEntry peek() {
+        return entries[0];
+    }
+
+    /**
+     * compare the new entry with entries within the queue, to find a spot for
+     * this new entry
+     * 
+     * @param entry
+     * @return runid of this entry
+     * @throws IOException
+     */
+    public int popAndReplace(FrameTupleAccessor fta, int tIndex) throws IOException {
+        ReferenceEntry entry = entries[0];
+        if (entry.getAccessor() == null) {
+            entry.setAccessor(new FrameTupleAccessor(ctx, recordDescriptor));
+        }
+        entry.getAccessor().reset(fta.getBuffer());
+        entry.setTupleIndex(tIndex);
+
+        add(entry);
+        return entry.getRunid();
+    }
+
+    /**
+     * Push entry into priority queue
+     * 
+     * @param e
+     *            the new Entry
+     */
+    private void add(ReferenceEntry e) {
+        ReferenceEntry min = entries[0];
+        int slot = (size >> 1) + (min.getRunid() >> 1);
+
+        ReferenceEntry curr = e;
+        while (!runAvail.isEmpty() && slot > 0) {
+            int c = 0;
+            if (!runAvail.get(entries[slot].getRunid())) {
+                // run of entries[slot] is exhausted, i.e. not available, curr
+                // wins
+                c = 1;
+            } else if (entries[slot].getAccessor() != null /*
+                                                            * entries[slot] is
+                                                            * not MIN value
+                                                            */
+                    && runAvail.get(curr.getRunid() /* curr run is available */)) {
+
+                if (curr.getAccessor() != null) {
+                    c = comparator.compare(entries[slot], curr);
+                } else {
+                    // curr is MIN value, wins
+                    c = 1;
+                }
+            }
+
+            if (c <= 0) { // curr lost
+                // entries[slot] swaps up
+                ReferenceEntry tmp = entries[slot];
+                entries[slot] = curr;
+                curr = tmp;// winner to pass up
+            }// else curr wins
+            slot >>= 1;
+        }
+        // set new entries[0]
+        entries[0] = curr;
+    }
+
+    /**
+     * Pop is called only when a run is exhausted
+     * 
+     * @return
+     */
+    public ReferenceEntry pop() {
+        ReferenceEntry min = entries[0];
+        runAvail.clear(min.getRunid());
+        add(min);
+        nItems--;
+        return min;
+    }
+
+    public boolean areRunsExhausted() {
+        return runAvail.isEmpty();
+    }
+
+    public int size() {
+        return nItems;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/SynchronizedBoundedBuffer.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/SynchronizedBoundedBuffer.java
new file mode 100644
index 0000000..f1d3d3c
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/SynchronizedBoundedBuffer.java
@@ -0,0 +1,68 @@
+/*
+ * 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.coreops.util;
+
+import java.util.Collection;
+
+public class SynchronizedBoundedBuffer<T> {
+    private static final int QUEUE_SIZE = 8192;
+    private Object[] buffer;
+    private int head;
+    private int tail;
+
+    public SynchronizedBoundedBuffer() {
+        buffer = new Object[QUEUE_SIZE];
+        head = 0;
+        tail = 0;
+    }
+
+    public synchronized void put(T o) throws InterruptedException {
+        while (full()) {
+            wait();
+        }
+        buffer[tail] = o;
+        tail = (tail + 1) % QUEUE_SIZE;
+        notifyAll();
+    }
+
+    public synchronized void putAll(Collection<? extends T> c) throws InterruptedException {
+        for (T o : c) {
+            while (full()) {
+                wait();
+            }
+            buffer[tail] = o;
+            tail = (tail + 1) % QUEUE_SIZE;
+        }
+        notifyAll();
+    }
+
+    public synchronized T get() throws InterruptedException {
+        while (empty()) {
+            wait();
+        }
+        T o = (T) buffer[head];
+        head = (head + 1) % QUEUE_SIZE;
+        notifyAll();
+        return o;
+    }
+
+    private boolean empty() {
+        return head == tail;
+    }
+
+    private boolean full() {
+        return (tail + 1) % QUEUE_SIZE == head;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/SynchronizedBoundedBufferDataReader.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/SynchronizedBoundedBufferDataReader.java
new file mode 100644
index 0000000..7599bdc
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/SynchronizedBoundedBufferDataReader.java
@@ -0,0 +1,45 @@
+/*
+ * 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.coreops.util;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class SynchronizedBoundedBufferDataReader implements IOpenableDataReader<Object[]> {
+    private SynchronizedBoundedBuffer<Object[]> queue;
+
+    public SynchronizedBoundedBufferDataReader(SynchronizedBoundedBuffer<Object[]> queue) {
+        this.queue = queue;
+    }
+
+    @Override
+    public Object[] readData() throws HyracksDataException {
+        try {
+            return queue.get();
+        } catch (InterruptedException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void close() {
+        queue = null;
+    }
+
+    @Override
+    public void open() {
+        // do nothing
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/SynchronizedBoundedBufferDataWriter.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/SynchronizedBoundedBufferDataWriter.java
new file mode 100644
index 0000000..311ea57
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/coreops/util/SynchronizedBoundedBufferDataWriter.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.coreops.util;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class SynchronizedBoundedBufferDataWriter implements IOpenableDataWriter<Object[]> {
+    private SynchronizedBoundedBuffer<Object[]> queue;
+
+    public SynchronizedBoundedBufferDataWriter(SynchronizedBoundedBuffer<Object[]> queue) {
+        this.queue = queue;
+    }
+
+    @Override
+    public void writeData(Object[] data) throws HyracksDataException {
+        try {
+            queue.put(data);
+        } catch (InterruptedException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        try {
+            queue.put(null);
+        } catch (InterruptedException e) {
+            throw new HyracksDataException(e);
+        }
+        queue = null;
+    }
+
+    @Override
+    public void open() {
+        // do nothing
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/dataflow/base/IOperatorDescriptorVisitor.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/dataflow/base/IOperatorDescriptorVisitor.java
new file mode 100644
index 0000000..c01d183
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/dataflow/base/IOperatorDescriptorVisitor.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.base;
+
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+
+public interface IOperatorDescriptorVisitor {
+    public void visit(IOperatorDescriptor op) throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/dataflow/util/PlanUtils.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/dataflow/util/PlanUtils.java
new file mode 100644
index 0000000..e22c60b
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/dataflow/util/PlanUtils.java
@@ -0,0 +1,40 @@
+/*
+ * 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.util;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.base.IOperatorDescriptorVisitor;
+
+public class PlanUtils {
+    public static void visit(JobSpecification spec, IOperatorDescriptorVisitor visitor) throws Exception {
+        Set<OperatorDescriptorId> seen = new HashSet<OperatorDescriptorId>();
+        for (IOperatorDescriptor op : spec.getOperatorMap().values()) {
+            visitOperator(visitor, seen, op);
+        }
+    }
+
+    private static void visitOperator(IOperatorDescriptorVisitor visitor, Set<OperatorDescriptorId> seen,
+            IOperatorDescriptor op) throws Exception {
+        if (!seen.contains(op)) {
+            visitor.visit(op);
+        }
+        seen.add(op.getOperatorId());
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/driver/CCDriver.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/driver/CCDriver.java
new file mode 100644
index 0000000..425f967
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/driver/CCDriver.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.driver;
+
+import org.kohsuke.args4j.CmdLineParser;
+
+import edu.uci.ics.hyracks.config.CCConfig;
+import edu.uci.ics.hyracks.controller.ClusterControllerService;
+
+public class CCDriver {
+    public static void main(String args[]) throws Exception {
+        CCConfig ccConfig = new CCConfig();
+        CmdLineParser cp = new CmdLineParser(ccConfig);
+        try {
+            cp.parseArgument(args);
+        } catch (Exception e) {
+            System.err.println(e.getMessage());
+            cp.printUsage(System.err);
+            return;
+        }
+        ClusterControllerService ccService = new ClusterControllerService(ccConfig);
+        ccService.start();
+        while (true) {
+            Thread.sleep(100000);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/driver/NCDriver.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/driver/NCDriver.java
new file mode 100644
index 0000000..657b304
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/driver/NCDriver.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.driver;
+
+import org.kohsuke.args4j.CmdLineParser;
+
+import edu.uci.ics.dcache.client.DCacheClient;
+import edu.uci.ics.dcache.client.DCacheClientConfig;
+import edu.uci.ics.hyracks.config.NCConfig;
+import edu.uci.ics.hyracks.controller.NodeControllerService;
+
+public class NCDriver {
+    public static void main(String args[]) throws Exception {
+        NCConfig ncConfig = new NCConfig();
+        CmdLineParser cp = new CmdLineParser(ncConfig);
+        try {
+            cp.parseArgument(args);
+        } catch (Exception e) {
+            System.err.println(e.getMessage());
+            cp.printUsage(System.err);
+            return;
+        }
+
+        DCacheClientConfig dccConfig = new DCacheClientConfig();
+        dccConfig.servers = ncConfig.dcacheClientServers;
+        dccConfig.serverLocal = ncConfig.dcacheClientServerLocal;
+        dccConfig.path = ncConfig.dcacheClientPath;
+
+        DCacheClient.get().init(dccConfig);
+
+        final NodeControllerService nService = new NodeControllerService(ncConfig);
+        nService.start();
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                try {
+                    nService.stop();
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        });
+        while (true) {
+            Thread.sleep(10000);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/ClasspathBasedHadoopClassFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/ClasspathBasedHadoopClassFactory.java
new file mode 100644
index 0000000..e15ef8c
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/ClasspathBasedHadoopClassFactory.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.hadoop.util;
+
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.Reducer;
+
+public class ClasspathBasedHadoopClassFactory implements IHadoopClassFactory {
+
+	@Override
+	public Mapper createMapper(String mapClassName) throws Exception {
+		Class clazz = loadClass(mapClassName);
+		return (Mapper)clazz.newInstance();
+	}
+
+	@Override
+	public Reducer createReducer(String reduceClassName) throws Exception {
+		Class clazz = loadClass(reduceClassName);
+		return (Reducer)clazz.newInstance();
+	}
+
+	@Override
+	public Class loadClass(String className) throws Exception {
+		Class clazz = Class.forName(className);
+		return clazz;
+	}
+
+	
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/DatatypeHelper.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/DatatypeHelper.java
new file mode 100644
index 0000000..199bd8d
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/DatatypeHelper.java
@@ -0,0 +1,117 @@
+/*
+ * 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.hadoop.util;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.JobConf;
+
+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;
+
+@SuppressWarnings("deprecation")
+public class DatatypeHelper {
+    private static final class WritableSerializerDeserializer<T extends Writable> implements ISerializerDeserializer<T> {
+        private static final long serialVersionUID = 1L;
+
+        private Class<T> clazz;
+
+        private WritableSerializerDeserializer(Class<T> clazz) {
+            this.clazz = clazz;
+        }
+
+        private T createInstance() throws HyracksDataException {
+            // TODO remove "if", create a new WritableInstanceOperations class
+            // that deals with Writables that don't have public constructors
+            if (NullWritable.class.equals(clazz)) {
+                return (T) NullWritable.get();
+            }
+            try {
+                return clazz.newInstance();
+            } catch (InstantiationException e) {
+                throw new HyracksDataException(e);
+            } catch (IllegalAccessException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        @Override
+        public T deserialize(DataInput in) throws HyracksDataException {
+            T o = createInstance();
+            try {
+                o.readFields(in);
+            } catch (IOException e) {
+                e.printStackTrace();
+                // throw new HyracksDataException(e);
+            }
+            return o;
+        }
+
+        @Override
+        public void serialize(T instance, DataOutput out) throws HyracksDataException {
+            try {
+                instance.write(out);
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    public static ISerializerDeserializer<? extends Writable> createSerializerDeserializer(
+            Class<? extends Writable> fClass) {
+        return new WritableSerializerDeserializer(fClass);
+    }
+
+    public static RecordDescriptor createKeyValueRecordDescriptor(Class<? extends Writable> keyClass,
+            Class<? extends Writable> valueClass) {
+        ISerializerDeserializer[] fields = new ISerializerDeserializer[2];
+        fields[0] = createSerializerDeserializer(keyClass);
+        fields[1] = createSerializerDeserializer(valueClass);
+        return new RecordDescriptor(fields);
+    }
+
+    public static RecordDescriptor createOneFieldRecordDescriptor(Class<? extends Writable> fieldClass) {
+        ISerializerDeserializer[] fields = new ISerializerDeserializer[1];
+        fields[0] = createSerializerDeserializer(fieldClass);
+        return new RecordDescriptor(fields);
+    }
+
+    public static JobConf hashMap2JobConf(HashMap<String, String> jobConfMap) {
+        JobConf jobConf;
+        synchronized (Configuration.class) {
+            jobConf = new JobConf();
+            for (Entry<String, String> entry : jobConfMap.entrySet()) {
+                jobConf.set(entry.getKey(), entry.getValue());
+            }
+        }
+        return jobConf;
+    }
+
+    public static HashMap<String, String> jobConf2HashMap(JobConf jobConf) {
+        HashMap<String, String> jobConfMap = new HashMap<String, String>();
+        for (Entry<String, String> entry : jobConf) {
+            jobConfMap.put(entry.getKey(), entry.getValue());
+        }
+        return jobConfMap;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/DuplicateKeyMapper.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/DuplicateKeyMapper.java
new file mode 100644
index 0000000..3d15d58
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/DuplicateKeyMapper.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.hadoop.util;
+
+import java.util.Properties;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.coreops.IMapper;
+
+public class DuplicateKeyMapper implements IMapper {
+
+    @Override
+    public void map(Object[] data, IDataWriter<Object[]> writer) throws HyracksDataException {
+        writer.writeData(new Object[] { data[0], data[1], data[0] });
+
+    }
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/HadoopAdapter.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/HadoopAdapter.java
new file mode 100644
index 0000000..a6edfc4
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/HadoopAdapter.java
@@ -0,0 +1,163 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.hadoop.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.util.StringUtils;
+
+import edu.uci.ics.hyracks.coreops.file.FileSplit;
+
+public class HadoopAdapter {
+
+	private static ClientProtocol namenode;
+	private static FileSystem fileSystem;
+	private static JobConf jobConf;
+	private static HadoopAdapter instance;
+	
+	public static final String DFS_DATA_DIR = "dfs.data.dir";
+	public static final String FS_DEFAULT_NAME = "fs.default.name";
+	public static final String DFS_REPLICATION = "dfs.replication";
+	
+	public static HadoopAdapter getInstance(String fileSystemURL){
+		if(instance == null){
+			jobConf = new JobConf(true);
+			String [] urlParts = parseURL(fileSystemURL);
+			jobConf.set(FS_DEFAULT_NAME, fileSystemURL);
+			instance = new HadoopAdapter(new InetSocketAddress(urlParts[1], Integer.parseInt(urlParts[2])));
+		}
+		return instance;
+	}
+	
+	public static JobConf getConf() {
+		return jobConf;
+	}
+
+	private HadoopAdapter (InetSocketAddress address){
+		try{
+			this.namenode = getNameNode(address);
+			fileSystem = FileSystem.get(jobConf);
+		}catch(IOException ioe){
+			ioe.printStackTrace();
+		}
+	}
+	
+	private static String[] parseURL(String urlString){
+		String[] urlComponents = urlString.split(":");
+		urlComponents[1] = urlComponents[1].substring(2);
+		return urlComponents;
+	}
+	
+	
+	public Map<String,List<HadoopFileSplit>> getInputSplits(String[] inputPaths){
+	    List<HadoopFileSplit> hadoopFileSplits = new ArrayList<HadoopFileSplit>();
+    	Path[] paths = new Path[inputPaths.length];
+    	int index =0;
+    	for(String inputPath : inputPaths){
+    		paths[index++] = new Path(StringUtils.unEscapeString(inputPaths[0]));
+    	}
+    	Map<String,List<HadoopFileSplit>> fileSplitInfo = getBlocksToRead(paths);
+    	return fileSplitInfo;
+	}
+	
+	private static Map<String,List<HadoopFileSplit>> getBlocksToRead(Path[] inputPaths){
+		Map<String,List<HadoopFileSplit>> hadoopFileSplitMap = new HashMap<String,List<HadoopFileSplit>>();
+		for (int i=0;i<inputPaths.length;i++){
+			try{
+				String absolutePathPrint = getAbsolutePath(inputPaths[i]);
+				FileStatus[] fileStatuses = namenode.getListing(absolutePathPrint);
+				for(int j=0;j<fileStatuses.length;j++){
+			    	Path filePath = fileStatuses[j].getPath();
+			    	String absolutePath = getAbsolutePath(filePath);
+			    	List<HadoopFileSplit> fileSplits = getFileBlocks(absolutePath,fileStatuses[j]);
+			    	if(fileSplits!=null && fileSplits.size() > 0){
+			    		hadoopFileSplitMap.put(absolutePath, fileSplits);
+			    	}	
+			    }		
+			   }catch(IOException ioe){
+				ioe.printStackTrace();
+		    }
+			
+		}
+		return hadoopFileSplitMap;
+	}
+	
+	private static ClientProtocol getNameNode(InetSocketAddress address) throws IOException{
+		return (ClientProtocol)getProtocol(ClientProtocol.class, address, new JobConf());
+	}
+	
+	private static String getAbsolutePath(Path path){
+		StringBuffer absolutePath = new StringBuffer();
+		List<String> ancestorPath = new ArrayList<String>();
+		Path pathElement=path;
+		while(pathElement != null){
+			ancestorPath.add(0, pathElement.getName());
+			pathElement = pathElement.getParent();
+		}
+		ancestorPath.remove(0);
+		for(String s : ancestorPath){
+			absolutePath.append("/");
+			absolutePath.append(s);
+		}
+		return new String(absolutePath);
+	}
+	
+	private static VersionedProtocol getProtocol(Class protocolClass, InetSocketAddress inetAddress, JobConf jobConf) throws IOException{
+		VersionedProtocol versionedProtocol = RPC.getProxy(protocolClass, ClientProtocol.versionID, inetAddress, jobConf);	
+		return versionedProtocol;
+	}
+	
+	private static List<HadoopFileSplit> getFileBlocks(String absolutePath,FileStatus fileStatus){
+		List<HadoopFileSplit> hadoopFileSplits = new ArrayList<HadoopFileSplit>();
+		try{
+			LocatedBlocks locatedBlocks = namenode.getBlockLocations(absolutePath, 0, fileStatus.getLen());
+			long blockSize = fileSystem.getBlockSize(new Path(absolutePath));
+			if(locatedBlocks !=null){
+	    		int index = 0;
+				for(LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()){
+					DatanodeInfo[] datanodeInfos = locatedBlock.getLocations(); // all datanodes having this block
+					String [] hostnames = new String[datanodeInfos.length];
+					int datanodeCount =0;
+					for(DatanodeInfo datanodeInfo : datanodeInfos){
+						hostnames[datanodeCount++] = datanodeInfo.getHostName();
+					}	
+					HadoopFileSplit hadoopFileSplit = new HadoopFileSplit(absolutePath,new Long(index * blockSize).longValue(),new Long(blockSize).longValue(),hostnames);
+			    	hadoopFileSplits.add(hadoopFileSplit);
+			    	index++;
+			    	}
+	    	}	
+		}catch(Exception e){
+			e.printStackTrace();
+		}
+		return hadoopFileSplits;
+	}
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/HadoopFileSplit.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/HadoopFileSplit.java
new file mode 100644
index 0000000..6f3810c
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/HadoopFileSplit.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.hadoop.util;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.fs.Path;
+
+public class HadoopFileSplit implements Serializable{
+	/**
+	 * 
+	 */
+	private static final long serialVersionUID = 1L;
+	private String filePath;
+    private long start;
+	private long length;
+    private String[] hosts;
+    
+    public HadoopFileSplit(String filePath, long start, long length, String[] hosts){
+    	this.filePath = filePath;
+    	this.start = start;
+    	this.length = length;
+    	this.hosts = hosts;
+    }
+
+	public String getFile() {
+		return filePath;
+	}
+
+	public void setFile(String file) {
+		this.filePath = file;
+	}
+
+	public long getStart() {
+		return start;
+	}
+
+	public void setStart(long start) {
+		this.start = start;
+	}
+
+	public long getLength() {
+		return length;
+	}
+
+	public void setLength(long length) {
+		this.length = length;
+	}
+
+	public String[] getHosts() {
+		return hosts;
+	}
+
+	public void setHosts(String[] hosts) {
+		this.hosts = hosts;
+	}
+	
+	public String toString(){
+		StringBuilder stringBuilder = new StringBuilder();
+		stringBuilder.append(filePath + " " + start + " " + length +  "\n");
+		for(String host : hosts){
+			stringBuilder.append(host);
+			stringBuilder.append(",");
+		}
+		return new String(stringBuilder);
+	}
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/IHadoopClassFactory.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/IHadoopClassFactory.java
new file mode 100644
index 0000000..c4f1d3c
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/IHadoopClassFactory.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.hadoop.util;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.Reducer;
+
+public interface IHadoopClassFactory extends Serializable{
+
+	public Mapper createMapper(String mapClassName) throws Exception;
+	
+	public Reducer createReducer(String reduceClassName) throws Exception;
+	
+	public Class loadClass(String className) throws Exception;
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/PreappendLongWritableMapper.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/PreappendLongWritableMapper.java
new file mode 100644
index 0000000..e561522
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/hadoop/util/PreappendLongWritableMapper.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.hadoop.util;
+
+import java.util.Properties;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.coreops.IMapper;
+
+public class PreappendLongWritableMapper implements IMapper {
+
+    @Override
+    public void map(Object[] data, IDataWriter<Object[]> writer) throws HyracksDataException {
+        writer.writeData(new Object[] { new LongWritable(0), new Text(String.valueOf(data[0])) });
+    }
+}
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/JobControl.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/JobControl.java
new file mode 100644
index 0000000..3259649
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/JobControl.java
@@ -0,0 +1,114 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.job;
+
+import java.rmi.RemoteException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.api.job.statistics.JobStatistics;
+import edu.uci.ics.hyracks.api.job.statistics.StageStatistics;
+import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
+
+public class JobControl {
+    private static final long serialVersionUID = 1L;
+
+    private final JobManager jobManager;
+
+    private final JobPlan jobPlan;
+
+    private final UUID jobId;
+
+    private final Map<UUID, StageProgress> stageProgressMap;
+
+    private final Set<UUID> completeStages;
+
+    private JobStatus jobStatus;
+
+    private JobStatistics jobStatistics;
+
+    public JobControl(JobManager jobManager, JobPlan jobPlan) throws RemoteException {
+        this.jobManager = jobManager;
+        this.jobPlan = jobPlan;
+        jobId = UUID.randomUUID();
+        stageProgressMap = new HashMap<UUID, StageProgress>();
+        completeStages = new HashSet<UUID>();
+        jobStatus = JobStatus.INITIALIZED;
+        jobStatistics = new JobStatistics();
+    }
+
+    public JobPlan getJobPlan() {
+        return jobPlan;
+    }
+
+    public UUID getJobId() {
+        return jobId;
+    }
+
+    public synchronized JobStatus getJobStatus() {
+        return jobStatus;
+    }
+
+    public Set<UUID> getCompletedStages() {
+        return completeStages;
+    }
+
+    public void setStatus(JobStatus status) {
+        jobStatus = status;
+    }
+
+    public StageProgress getStageProgress(int stageId) {
+        return stageProgressMap.get(stageId);
+    }
+
+    public void setStageProgress(UUID stageId, StageProgress stageProgress) {
+        stageProgressMap.put(stageId, stageProgress);
+    }
+
+    public synchronized void notifyStageletComplete(UUID stageId, String nodeId, StageletStatistics ss)
+            throws Exception {
+        StageProgress stageProgress = stageProgressMap.get(stageId);
+        stageProgress.markNodeComplete(nodeId);
+        StageStatistics stageStatistics = stageProgress.getStageStatistics();
+        stageStatistics.addStageletStatistics(ss);
+        if (stageProgress.stageComplete()) {
+            jobStatistics.addStageStatistics(stageStatistics);
+            stageProgressMap.remove(stageId);
+            completeStages.add(stageId);
+            jobManager.advanceJob(this);
+        }
+    }
+
+    public synchronized JobStatistics waitForCompletion() throws Exception {
+        while (jobStatus != JobStatus.TERMINATED) {
+            wait();
+        }
+        return jobStatistics;
+    }
+
+    public synchronized void notifyJobComplete() {
+        jobStatus = JobStatus.TERMINATED;
+        notifyAll();
+    }
+
+    public JobStatistics getJobStatistics() {
+        return jobStatistics;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/JobManager.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/JobManager.java
new file mode 100644
index 0000000..f976bb9
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/JobManager.java
@@ -0,0 +1,352 @@
+/*
+ * 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.job;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.Vector;
+import java.util.concurrent.Semaphore;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.Endpoint;
+import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.controller.INodeController;
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.job.JobStage;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.api.job.statistics.JobStatistics;
+import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
+import edu.uci.ics.hyracks.controller.ClusterControllerService;
+import edu.uci.ics.hyracks.dataflow.base.IOperatorDescriptorVisitor;
+import edu.uci.ics.hyracks.dataflow.util.PlanUtils;
+
+public class JobManager {
+    private static final Logger LOGGER = Logger.getLogger(JobManager.class.getName());
+    private ClusterControllerService ccs;
+
+    private final Map<UUID, JobControl> jobMap;
+
+    public JobManager(ClusterControllerService ccs) {
+        this.ccs = ccs;
+        jobMap = new HashMap<UUID, JobControl>();
+    }
+
+    public synchronized UUID createJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+        JobPlanBuilder builder = new JobPlanBuilder();
+        builder.init(jobSpec, jobFlags);
+        JobControl jc = new JobControl(this, builder.plan());
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(jc.getJobPlan().toString());
+        }
+        jobMap.put(jc.getJobId(), jc);
+
+        return jc.getJobId();
+    }
+
+    public synchronized void start(UUID jobId) throws Exception {
+        JobControl jobControlImpl = jobMap.get(jobId);
+        LOGGER
+                .info("Starting job: " + jobControlImpl.getJobId() + ", Current status: "
+                        + jobControlImpl.getJobStatus());
+        if (jobControlImpl.getJobStatus() != JobStatus.INITIALIZED) {
+            return;
+        }
+        jobControlImpl.getJobStatistics().setStartTime(new Date());
+        jobControlImpl.setStatus(JobStatus.RUNNING);
+        schedule(jobControlImpl);
+    }
+
+    public synchronized void advanceJob(JobControl jobControlImpl) throws Exception {
+        schedule(jobControlImpl);
+    }
+
+    private void schedule(JobControl jobControlImpl) throws Exception {
+        JobPlan plan = jobControlImpl.getJobPlan();
+        JobStage endStage = plan.getEndStage();
+
+        Set<UUID> completedStages = jobControlImpl.getCompletedStages();
+        List<JobStage> runnableStages = new ArrayList<JobStage>();
+        findRunnableStages(endStage, runnableStages, completedStages, new HashSet<UUID>());
+        if (runnableStages.size() == 1 && runnableStages.get(0).getTasks().isEmpty()) {
+            LOGGER.info("Job " + jobControlImpl.getJobId() + " complete");
+            jobControlImpl.getJobStatistics().setEndTime(new Date());
+            cleanUp(jobControlImpl);
+            jobControlImpl.notifyJobComplete();
+        } else {
+            for (JobStage s : runnableStages) {
+                if (s.isStarted()) {
+                    continue;
+                }
+                startStage(jobControlImpl, s);
+            }
+        }
+    }
+
+    private void cleanUp(JobControl jc) {
+        jobMap.remove(jc.getJobId());
+        ccs.notifyJobComplete(jc.getJobId());
+    }
+
+    private void startStage(JobControl jc, JobStage stage) throws Exception {
+        stage.setStarted();
+        Set<String> candidateNodes = deploy(jc, stage);
+        for (String nodeId : candidateNodes) {
+            ccs.lookupNode(nodeId).startStage(jc.getJobId(), stage.getId());
+        }
+    }
+
+    private void findRunnableStages(JobStage s, List<JobStage> runnableStages, Set<UUID> completedStages, Set<UUID> seen) {
+        boolean runnable = true;
+        if (seen.contains(s.getId())) {
+            return;
+        }
+        seen.add(s.getId());
+        for (JobStage dep : s.getDependencies()) {
+            boolean depComplete = completedStages.contains(dep.getId());
+            runnable = runnable && depComplete;
+            if (!depComplete) {
+                findRunnableStages(dep, runnableStages, completedStages, seen);
+            }
+        }
+        if (runnable) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Runnable stage: " + s);
+            }
+            runnableStages.add(s);
+        }
+    }
+
+    private Set<String> deploy(JobControl jc, JobStage stage) throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Deploying: " + stage);
+        }
+        Set<String> candidateNodes = plan(jc, stage);
+        StageProgress stageProgress = new StageProgress(stage.getId());
+        stageProgress.addPendingNodes(candidateNodes);
+        Map<PortInstanceId, Endpoint> globalPortMap = runRemote(new Phase1Installer(jc, stage),
+                new PortMapMergingAccumulator(), candidateNodes);
+        runRemote(new Phase2Installer(jc, stage, globalPortMap), null, candidateNodes);
+        runRemote(new Phase3Installer(jc, stage), null, candidateNodes);
+        jc.setStageProgress(stage.getId(), stageProgress);
+        return candidateNodes;
+    }
+
+    private interface RemoteOp<T> {
+        public T execute(INodeController node) throws Exception;
+    }
+
+    private interface Accumulator<T, R> {
+        public void accumulate(T o);
+
+        public R getResult();
+    }
+
+    private static class Phase1Installer implements RemoteOp<Map<PortInstanceId, Endpoint>> {
+        private JobControl jc;
+        private JobStage stage;
+
+        public Phase1Installer(JobControl jc, JobStage stage) {
+            this.jc = jc;
+            this.stage = stage;
+        }
+
+        @Override
+        public Map<PortInstanceId, Endpoint> execute(INodeController node) throws Exception {
+            return node.initializeJobletPhase1(jc.getJobId(), jc.getJobPlan(), stage);
+        }
+
+        @Override
+        public String toString() {
+            return jc.getJobId() + " Distribution Phase 1";
+        }
+    }
+
+    private static class Phase2Installer implements RemoteOp<Void> {
+        private JobControl jc;
+        private JobStage stage;
+        private Map<PortInstanceId, Endpoint> globalPortMap;
+
+        public Phase2Installer(JobControl jc, JobStage stage, Map<PortInstanceId, Endpoint> globalPortMap) {
+            this.jc = jc;
+            this.stage = stage;
+            this.globalPortMap = globalPortMap;
+        }
+
+        @Override
+        public Void execute(INodeController node) throws Exception {
+            node.initializeJobletPhase2(jc.getJobId(), jc.getJobPlan(), stage, globalPortMap);
+            return null;
+        }
+
+        @Override
+        public String toString() {
+            return jc.getJobId() + " Distribution Phase 2";
+        }
+    }
+
+    private static class Phase3Installer implements RemoteOp<Void> {
+        private JobControl jc;
+        private JobStage stage;
+
+        public Phase3Installer(JobControl jc, JobStage stage) {
+            this.jc = jc;
+            this.stage = stage;
+        }
+
+        @Override
+        public Void execute(INodeController node) throws Exception {
+            node.commitJobletInitialization(jc.getJobId(), jc.getJobPlan(), stage);
+            return null;
+        }
+
+        @Override
+        public String toString() {
+            return jc.getJobId() + " Distribution Phase 3";
+        }
+    }
+
+    private static class PortMapMergingAccumulator implements
+            Accumulator<Map<PortInstanceId, Endpoint>, Map<PortInstanceId, Endpoint>> {
+        Map<PortInstanceId, Endpoint> portMap = new HashMap<PortInstanceId, Endpoint>();
+
+        @Override
+        public void accumulate(Map<PortInstanceId, Endpoint> o) {
+            portMap.putAll(o);
+        }
+
+        @Override
+        public Map<PortInstanceId, Endpoint> getResult() {
+            return portMap;
+        }
+    }
+
+    private <T, R> R runRemote(final RemoteOp<T> remoteOp, final Accumulator<T, R> accumulator,
+            Set<String> candidateNodes) throws Exception {
+        LOGGER.log(Level.INFO, remoteOp + " : " + candidateNodes);
+
+        final Semaphore installComplete = new Semaphore(candidateNodes.size());
+        final List<Exception> errors = new Vector<Exception>();
+        for (final String nodeId : candidateNodes) {
+            final INodeController node = ccs.lookupNode(nodeId);
+
+            installComplete.acquire();
+            Runnable remoteRunner = new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        T t = remoteOp.execute(node);
+                        if (accumulator != null) {
+                            synchronized (accumulator) {
+                                accumulator.accumulate(t);
+                            }
+                        }
+                    } catch (Exception e) {
+                        errors.add(e);
+                    } finally {
+                        installComplete.release();
+                    }
+                }
+            };
+
+            ccs.getExecutor().execute(remoteRunner);
+        }
+        installComplete.acquire(candidateNodes.size());
+        if (!errors.isEmpty()) {
+            throw errors.get(0);
+        }
+        return accumulator == null ? null : accumulator.getResult();
+    }
+
+    private Set<String> plan(JobControl jc, JobStage stage) throws Exception {
+        LOGGER.log(Level.INFO, String.valueOf(jc.getJobId()) + ": Planning");
+
+        final Set<OperatorDescriptorId> opSet = new HashSet<OperatorDescriptorId>();
+        for (ActivityNodeId t : stage.getTasks()) {
+            opSet.add(jc.getJobPlan().getActivityNodeMap().get(t).getOwner().getOperatorId());
+        }
+
+        final Set<String> candidateNodes = new HashSet<String>();
+
+        IOperatorDescriptorVisitor visitor = new IOperatorDescriptorVisitor() {
+            @Override
+            public void visit(IOperatorDescriptor op) throws Exception {
+                if (!opSet.contains(op.getOperatorId())) {
+                    return;
+                }
+                String[] partitions = op.getPartitions();
+                if (partitions == null) {
+                    PartitionConstraint pc = op.getPartitionConstraint();
+                    LocationConstraint[] lcs = pc.getLocationConstraints();
+                    String[] assignment = new String[lcs.length];
+                    for (int i = 0; i < lcs.length; ++i) {
+                        String nodeId = ((AbsoluteLocationConstraint) lcs[i]).getLocationId();
+                        assignment[i] = nodeId;
+                    }
+                    op.setPartitions(assignment);
+                    partitions = assignment;
+                }
+                for (String p : partitions) {
+                    candidateNodes.add(p);
+                }
+            }
+        };
+
+        PlanUtils.visit(jc.getJobPlan().getJobSpecification(), visitor);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(stage + " Candidate nodes: " + candidateNodes);
+        }
+        return candidateNodes;
+    }
+
+    public synchronized void notifyStageletComplete(UUID jobId, UUID stageId, String nodeId,
+            StageletStatistics statistics) throws Exception {
+        JobControl jc = jobMap.get(jobId);
+        if (jc != null) {
+            jc.notifyStageletComplete(stageId, nodeId, statistics);
+        }
+    }
+
+    public synchronized JobStatus getJobStatus(UUID jobId) {
+        JobControl jc = jobMap.get(jobId);
+        return jc.getJobStatus();
+    }
+
+    public JobStatistics waitForCompletion(UUID jobId) throws Exception {
+        JobControl jc;
+        synchronized (this) {
+            jc = jobMap.get(jobId);
+        }
+        if (jc != null) {
+            return jc.waitForCompletion();
+        }
+        return null;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/JobPlanBuilder.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/JobPlanBuilder.java
new file mode 100644
index 0000000..61cf70f
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/JobPlanBuilder.java
@@ -0,0 +1,235 @@
+/*
+ * 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.job;
+
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.job.JobStage;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.dataflow.base.IOperatorDescriptorVisitor;
+import edu.uci.ics.hyracks.dataflow.util.PlanUtils;
+
+public class JobPlanBuilder implements IActivityGraphBuilder {
+    private static final Logger LOGGER = Logger.getLogger(JobPlanBuilder.class.getName());
+
+    private JobPlan plan;
+
+    @Override
+    public void addBlockingEdge(IActivityNode blocker, IActivityNode blocked) {
+        addToValueSet(plan.getBlocker2BlockedMap(), blocker.getActivityNodeId(), blocked.getActivityNodeId());
+        addToValueSet(plan.getBlocked2BlockerMap(), blocked.getActivityNodeId(), blocker.getActivityNodeId());
+    }
+
+    @Override
+    public void addSourceEdge(int operatorInputIndex, IActivityNode task, int taskInputIndex) {
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("Adding source edge: " + task.getOwner().getOperatorId() + ":" + operatorInputIndex + " -> "
+                    + task.getActivityNodeId() + ":" + taskInputIndex);
+        }
+        insertIntoIndexedMap(plan.getTaskInputMap(), task.getActivityNodeId(), taskInputIndex, operatorInputIndex);
+        insertIntoIndexedMap(plan.getOperatorInputMap(), task.getOwner().getOperatorId(), operatorInputIndex, task
+                .getActivityNodeId());
+    }
+
+    @Override
+    public void addTargetEdge(int operatorOutputIndex, IActivityNode task, int taskOutputIndex) {
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("Adding target edge: " + task.getOwner().getOperatorId() + ":" + operatorOutputIndex + " -> "
+                    + task.getActivityNodeId() + ":" + taskOutputIndex);
+        }
+        insertIntoIndexedMap(plan.getTaskOutputMap(), task.getActivityNodeId(), taskOutputIndex, operatorOutputIndex);
+        insertIntoIndexedMap(plan.getOperatorOutputMap(), task.getOwner().getOperatorId(), operatorOutputIndex, task
+                .getActivityNodeId());
+    }
+
+    @Override
+    public void addTask(IActivityNode task) {
+        plan.getActivityNodeMap().put(task.getActivityNodeId(), task);
+        addToValueSet(plan.getOperatorTaskMap(), task.getOwner().getOperatorId(), task.getActivityNodeId());
+    }
+
+    private <K, V> void addToValueSet(Map<K, Set<V>> map, K n1, V n2) {
+        Set<V> targets = map.get(n1);
+        if (targets == null) {
+            targets = new HashSet<V>();
+            map.put(n1, targets);
+        }
+        targets.add(n2);
+    }
+
+    private <T> void extend(List<T> list, int index) {
+        int n = list.size();
+        for (int i = n; i <= index; ++i) {
+            list.add(null);
+        }
+    }
+
+    private Pair<ActivityNodeId, ActivityNodeId> findMergePair(JobPlan plan, JobSpecification spec, Set<JobStage> eqSets) {
+        Map<ActivityNodeId, IActivityNode> activityNodeMap = plan.getActivityNodeMap();
+        for (JobStage eqSet : eqSets) {
+            for (ActivityNodeId t : eqSet.getTasks()) {
+                IOperatorDescriptor owner = activityNodeMap.get(t).getOwner();
+                List<Integer> inputList = plan.getTaskInputMap().get(t);
+                if (inputList != null) {
+                    for (Integer idx : inputList) {
+                        IConnectorDescriptor conn = spec.getInputConnectorDescriptor(owner, idx);
+                        OperatorDescriptorId producerId = spec.getProducer(conn).getOperatorId();
+                        int producerOutputIndex = spec.getProducerOutputIndex(conn);
+                        ActivityNodeId inTask = plan.getOperatorOutputMap().get(producerId).get(producerOutputIndex);
+                        if (!eqSet.getTasks().contains(inTask)) {
+                            return new Pair<ActivityNodeId, ActivityNodeId>(t, inTask);
+                        }
+                    }
+                }
+                List<Integer> outputList = plan.getTaskOutputMap().get(t);
+                if (outputList != null) {
+                    for (Integer idx : outputList) {
+                        IConnectorDescriptor conn = spec.getOutputConnectorDescriptor(owner, idx);
+                        OperatorDescriptorId consumerId = spec.getConsumer(conn).getOperatorId();
+                        int consumerInputIndex = spec.getConsumerInputIndex(conn);
+                        ActivityNodeId outTask = plan.getOperatorInputMap().get(consumerId).get(consumerInputIndex);
+                        if (!eqSet.getTasks().contains(outTask)) {
+                            return new Pair<ActivityNodeId, ActivityNodeId>(t, outTask);
+                        }
+                    }
+                }
+            }
+        }
+        return null;
+    }
+
+    private JobStage inferStages() throws Exception {
+        JobSpecification spec = plan.getJobSpecification();
+
+        /*
+         * Build initial equivalence sets map. We create a map such that for
+         * each IOperatorTask, t -> { t }
+         */
+        Map<ActivityNodeId, JobStage> stageMap = new HashMap<ActivityNodeId, JobStage>();
+        Set<JobStage> stages = new HashSet<JobStage>();
+        for (Set<ActivityNodeId> taskIds : plan.getOperatorTaskMap().values()) {
+            for (ActivityNodeId taskId : taskIds) {
+                Set<ActivityNodeId> eqSet = new HashSet<ActivityNodeId>();
+                eqSet.add(taskId);
+                JobStage stage = new JobStage(eqSet);
+                stageMap.put(taskId, stage);
+                stages.add(stage);
+            }
+        }
+
+        boolean changed = true;
+        while (changed) {
+            changed = false;
+            Pair<ActivityNodeId, ActivityNodeId> pair = findMergePair(plan, spec, stages);
+            if (pair != null) {
+                merge(stageMap, stages, pair.first, pair.second);
+                changed = true;
+            }
+        }
+
+        JobStage endStage = new JobStage(new HashSet<ActivityNodeId>());
+        Map<ActivityNodeId, Set<ActivityNodeId>> blocker2BlockedMap = plan.getBlocker2BlockedMap();
+        for (JobStage s : stages) {
+            endStage.addDependency(s);
+            s.addDependent(endStage);
+            Set<JobStage> blockedStages = new HashSet<JobStage>();
+            for (ActivityNodeId t : s.getTasks()) {
+                Set<ActivityNodeId> blockedTasks = blocker2BlockedMap.get(t);
+                if (blockedTasks != null) {
+                    for (ActivityNodeId bt : blockedTasks) {
+                        blockedStages.add(stageMap.get(bt));
+                    }
+                }
+            }
+            for (JobStage bs : blockedStages) {
+                bs.addDependency(s);
+                s.addDependent(bs);
+            }
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Inferred " + (stages.size() + 1) + " stages");
+            for (JobStage s : stages) {
+                LOGGER.info(s.toString());
+            }
+            LOGGER.info("SID: ENDSTAGE");
+        }
+        return endStage;
+    }
+
+    public void init(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
+        plan = new JobPlan(jobSpec, jobFlags);
+    }
+
+    private <K, V> void insertIntoIndexedMap(Map<K, List<V>> map, K key, int index, V value) {
+        List<V> vList = map.get(key);
+        if (vList == null) {
+            vList = new ArrayList<V>();
+            map.put(key, vList);
+        }
+        extend(vList, index);
+        vList.set(index, value);
+    }
+
+    private void merge(Map<ActivityNodeId, JobStage> eqSetMap, Set<JobStage> eqSets, ActivityNodeId t1,
+            ActivityNodeId t2) {
+        JobStage stage1 = eqSetMap.get(t1);
+        Set<ActivityNodeId> s1 = stage1.getTasks();
+        JobStage stage2 = eqSetMap.get(t2);
+        Set<ActivityNodeId> s2 = stage2.getTasks();
+
+        Set<ActivityNodeId> mergedSet = new HashSet<ActivityNodeId>();
+        mergedSet.addAll(s1);
+        mergedSet.addAll(s2);
+
+        eqSets.remove(stage1);
+        eqSets.remove(stage2);
+        JobStage mergedStage = new JobStage(mergedSet);
+        eqSets.add(mergedStage);
+
+        for (ActivityNodeId t : mergedSet) {
+            eqSetMap.put(t, mergedStage);
+        }
+    }
+
+    public JobPlan plan() throws Exception {
+        PlanUtils.visit(plan.getJobSpecification(), new IOperatorDescriptorVisitor() {
+            @Override
+            public void visit(IOperatorDescriptor op) throws Exception {
+                op.contributeTaskGraph(JobPlanBuilder.this);
+            }
+        });
+        JobStage endStage = inferStages();
+        plan.setEndStage(endStage);
+
+        return plan;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/Joblet.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/Joblet.java
new file mode 100644
index 0000000..5f38e56
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/Joblet.java
@@ -0,0 +1,95 @@
+/*
+ * 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.job;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.Executor;
+
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
+import edu.uci.ics.hyracks.controller.NodeControllerService;
+
+public class Joblet {
+    private static final long serialVersionUID = 1L;
+
+    private final NodeControllerService nodeController;
+
+    private final UUID jobId;
+
+    private final Map<UUID, Stagelet> stageletMap;
+
+    private final Map<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>> envMap;
+
+    public Joblet(NodeControllerService nodeController, UUID jobId) throws Exception {
+        this.nodeController = nodeController;
+        this.jobId = jobId;
+        stageletMap = new HashMap<UUID, Stagelet>();
+        envMap = new HashMap<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>>();
+    }
+
+    public UUID getJobId() {
+        return jobId;
+    }
+
+    public IOperatorEnvironment getEnvironment(IOperatorDescriptor hod, int partition) {
+        if (!envMap.containsKey(hod.getOperatorId())) {
+            envMap.put(hod.getOperatorId(), new HashMap<Integer, IOperatorEnvironment>());
+        }
+        Map<Integer, IOperatorEnvironment> opEnvMap = envMap.get(hod.getOperatorId());
+        if (!opEnvMap.containsKey(partition)) {
+            opEnvMap.put(partition, new OperatorEnvironmentImpl());
+        }
+        return opEnvMap.get(partition);
+    }
+
+    private static final class OperatorEnvironmentImpl implements IOperatorEnvironment {
+        private final Map<String, Object> map;
+
+        public OperatorEnvironmentImpl() {
+            map = new HashMap<String, Object>();
+        }
+
+        @Override
+        public Object get(String name) {
+            return map.get(name);
+        }
+
+        @Override
+        public void set(String name, Object value) {
+            map.put(name, value);
+        }
+    }
+
+    public void setStagelet(UUID stageId, Stagelet stagelet) {
+        stageletMap.put(stageId, stagelet);
+    }
+
+    public Stagelet getStagelet(UUID stageId) throws Exception {
+        return stageletMap.get(stageId);
+    }
+
+    public Executor getExecutor() {
+        return nodeController.getExecutor();
+    }
+
+    public synchronized void notifyStageletComplete(UUID stageId, StageletStatistics stats) throws Exception {
+        stageletMap.remove(stageId);
+        nodeController.notifyStageComplete(jobId, stageId, stats);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/StageProgress.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/StageProgress.java
new file mode 100644
index 0000000..3357012
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/StageProgress.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.job;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.job.statistics.StageStatistics;
+
+public class StageProgress {
+    private final UUID stageId;
+
+    private final Set<String> pendingNodes;
+
+    private final StageStatistics stageStatistics;
+
+    public StageProgress(UUID stageId) {
+        this.stageId = stageId;
+        pendingNodes = new HashSet<String>();
+        stageStatistics = new StageStatistics();
+        stageStatistics.setStageId(stageId);
+    }
+
+    public UUID getStageId() {
+        return stageId;
+    }
+
+    public void addPendingNodes(Set<String> nodes) {
+        pendingNodes.addAll(nodes);
+    }
+
+    public void markNodeComplete(String nodeId) {
+        pendingNodes.remove(nodeId);
+    }
+
+    public boolean stageComplete() {
+        return pendingNodes.isEmpty();
+    }
+
+    public StageStatistics getStageStatistics() {
+        return stageStatistics;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/Stagelet.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/Stagelet.java
new file mode 100644
index 0000000..e82fa5b
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/job/Stagelet.java
@@ -0,0 +1,136 @@
+/*
+ * 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.job;
+
+import java.rmi.RemoteException;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.Endpoint;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.OperatorInstanceId;
+import edu.uci.ics.hyracks.api.job.statistics.StageletStatistics;
+import edu.uci.ics.hyracks.runtime.OperatorRunnable;
+
+public class Stagelet {
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOGGER = Logger.getLogger(Stagelet.class.getName());
+
+    private final Joblet joblet;
+
+    private final UUID stageId;
+
+    private final Map<OperatorInstanceId, OperatorRunnable> honMap;
+
+    private List<Endpoint> endpointList;
+
+    private boolean started;
+
+    private final Set<OperatorInstanceId> pendingOperators;
+
+    private final StageletStatistics stats;
+
+    public Stagelet(Joblet joblet, UUID stageId, String nodeId) throws RemoteException {
+        this.joblet = joblet;
+        this.stageId = stageId;
+        pendingOperators = new HashSet<OperatorInstanceId>();
+        started = false;
+        honMap = new HashMap<OperatorInstanceId, OperatorRunnable>();
+        stats = new StageletStatistics();
+        stats.setNodeId(nodeId);
+    }
+
+    public void setOperator(OperatorDescriptorId odId, int partition, OperatorRunnable hon) {
+        honMap.put(new OperatorInstanceId(odId, partition), hon);
+    }
+
+    public Map<OperatorInstanceId, OperatorRunnable> getOperatorMap() {
+        return honMap;
+    }
+
+    public void setEndpointList(List<Endpoint> endpointList) {
+        this.endpointList = endpointList;
+    }
+
+    public List<Endpoint> getEndpointList() {
+        return endpointList;
+    }
+
+    public synchronized void start() throws Exception {
+        if (started) {
+            throw new Exception("Joblet already started");
+        }
+        started = true;
+        stats.setStartTime(new Date());
+        notifyAll();
+    }
+
+    public void installRunnable(final OperatorInstanceId opIId) {
+        pendingOperators.add(opIId);
+        final OperatorRunnable hon = honMap.get(opIId);
+        joblet.getExecutor().execute(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    waitUntilStarted();
+                } catch (InterruptedException e) {
+                    e.printStackTrace();
+                    return;
+                }
+                try {
+                    LOGGER.log(Level.INFO, "Starting runnable for operator: " + joblet.getJobId() + ":" + stageId + ":"
+                            + opIId.getOperatorId() + ":" + opIId.getPartition());
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+                try {
+                    hon.run();
+                } finally {
+                    notifyOperatorCompletion(opIId);
+                }
+            }
+        });
+    }
+
+    protected synchronized void notifyOperatorCompletion(OperatorInstanceId opIId) {
+        pendingOperators.remove(opIId);
+        if (pendingOperators.isEmpty()) {
+            stats.setEndTime(new Date());
+            try {
+                joblet.notifyStageletComplete(stageId, stats);
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    private synchronized void waitUntilStarted() throws InterruptedException {
+        while (!started) {
+            wait();
+        }
+    }
+
+    public StageletStatistics getStatistics() {
+        return stats;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/resources/ResourceManager.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/resources/ResourceManager.java
new file mode 100644
index 0000000..38be3ff
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/resources/ResourceManager.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.resources;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public final class ResourceManager {
+    private final HyracksContext ctx;
+
+    public ResourceManager(HyracksContext ctx) {
+        this.ctx = ctx;
+    }
+
+    public ByteBuffer allocateFrame() {
+        return ByteBuffer.allocate(ctx.getFrameSize());
+    }
+
+    public File createFile(String prefix, String suffix) throws IOException {
+        return File.createTempFile(prefix, suffix);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/runtime/OperatorRunnable.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/runtime/OperatorRunnable.java
new file mode 100644
index 0000000..5964d07
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/runtime/OperatorRunnable.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.runtime;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.context.HyracksContext;
+
+public class OperatorRunnable implements Runnable {
+    private IOperatorNodePushable opNode;
+    private IFrameReader reader;
+    private ByteBuffer buffer;
+
+    public OperatorRunnable(HyracksContext ctx, IOperatorNodePushable opNode) {
+        this.opNode = opNode;
+        buffer = ctx.getResourceManager().allocateFrame();
+    }
+
+    public void setFrameWriter(int index, IFrameWriter writer) {
+        opNode.setFrameWriter(index, writer);
+    }
+
+    public void setFrameReader(IFrameReader reader) {
+        this.reader = reader;
+    }
+
+    @Override
+    public void run() {
+        try {
+            opNode.open();
+            if (reader != null) {
+                reader.open();
+                while (reader.nextFrame(buffer)) {
+                    buffer.flip();
+                    opNode.nextFrame(buffer);
+                    buffer.compact();
+                }
+                reader.close();
+            }
+            opNode.close();
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/service/AbstractService.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/service/AbstractService.java
new file mode 100644
index 0000000..fcb6ea4
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/service/AbstractService.java
@@ -0,0 +1,18 @@
+/*
+ * 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.service;
+
+public abstract class AbstractService implements IService {
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/service/IService.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/service/IService.java
new file mode 100644
index 0000000..edfdc83
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/service/IService.java
@@ -0,0 +1,20 @@
+/*
+ * 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.service;
+
+import edu.uci.ics.hyracks.util.ILifeCycle;
+
+public interface IService extends ILifeCycle {
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/util/ILifeCycle.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/util/ILifeCycle.java
new file mode 100644
index 0000000..e526fa6
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/util/ILifeCycle.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.util;
+
+public interface ILifeCycle {
+    public void start() throws Exception;
+
+    public void stop() throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/util/PropertiesAccess.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/util/PropertiesAccess.java
new file mode 100644
index 0000000..3e01dc9
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/util/PropertiesAccess.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.util;
+
+import java.util.Properties;
+
+public class PropertiesAccess {
+    public static boolean safeBoolean(Properties props, String pName, boolean defaultValue) {
+        String pValue = props.getProperty(pName);
+        if (pValue == null) {
+            return defaultValue;
+        }
+        try {
+            return Boolean.parseBoolean(pValue);
+        } catch (Exception e) {
+            return defaultValue;
+        }
+    }
+
+    public static String safeString(Properties props, String pName, String defaultValue) {
+        String pValue = props.getProperty(pName);
+        if (pValue == null) {
+            return defaultValue;
+        }
+        return pValue;
+    }
+
+    public static int safeInt(Properties props, String pName, int defaultValue) {
+        String pValue = props.getProperty(pName);
+        if (pValue == null) {
+            return defaultValue;
+        }
+        try {
+            return Integer.parseInt(pValue);
+        } catch (Exception e) {
+            return defaultValue;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/util/ReflectionUtils.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/util/ReflectionUtils.java
new file mode 100644
index 0000000..bb391d7
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/util/ReflectionUtils.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.util;
+
+public class ReflectionUtils {
+    public static <T> T createInstance(Class<? extends T> klass) {
+        T instance = null;
+        try {
+            instance = klass.newInstance();
+        } catch (InstantiationException e) {
+            throw new RuntimeException(e);
+        } catch (IllegalAccessException e) {
+            throw new RuntimeException(e);
+        }
+        return instance;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/java/edu/uci/ics/hyracks/web/WebServer.java b/hyracks-core/src/main/java/edu/uci/ics/hyracks/web/WebServer.java
new file mode 100644
index 0000000..b6ab52b
--- /dev/null
+++ b/hyracks-core/src/main/java/edu/uci/ics/hyracks/web/WebServer.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.web;
+
+import org.eclipse.jetty.server.Connector;
+import org.eclipse.jetty.server.Handler;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.handler.ContextHandlerCollection;
+import org.eclipse.jetty.server.nio.SelectChannelConnector;
+
+public class WebServer {
+    private Server server;
+    private SelectChannelConnector connector;
+
+    public WebServer(Handler[] handlers) throws Exception {
+        server = new Server();
+
+        connector = new SelectChannelConnector();
+
+        server.setConnectors(new Connector[] { connector });
+
+        ContextHandlerCollection handler = new ContextHandlerCollection();
+        handler.setHandlers(handlers);
+        server.setHandler(handler);
+    }
+
+    public void setPort(int port) {
+        connector.setPort(port);
+    }
+
+    public void start() throws Exception {
+        server.start();
+    }
+
+    public void stop() throws Exception {
+        server.stop();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/main/scripts/run.cmd b/hyracks-core/src/main/scripts/run.cmd
new file mode 100644
index 0000000..2df8467
--- /dev/null
+++ b/hyracks-core/src/main/scripts/run.cmd
@@ -0,0 +1,62 @@
+@ECHO OFF
+SETLOCAL
+
+:: Licensed to the Apache Software Foundation (ASF) under one or more
+:: contributor license agreements.  See the NOTICE file distributed with
+:: this work for additional information regarding copyright ownership.
+:: The ASF licenses this file to You under the Apache License, Version 2.0
+:: (the "License"); you may not use this file except in compliance with
+:: the License.  You may obtain a copy of the License at
+::
+::     http://www.apache.org/licenses/LICENSE-2.0
+::
+:: Unless required by applicable law or agreed to in writing, software
+:: distributed under the License is distributed on an "AS IS" BASIS,
+:: WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+:: See the License for the specific language governing permissions and
+:: limitations under the License.
+
+:: JAVA classpath
+:: Use the local variable CLASSPATH to add custom entries (e.g. JDBC drivers) to
+:: the classpath. Separate multiple paths with ":". Enclose the value
+:: in double quotes. Adding additional files or locations on separate
+:: lines makes things clearer.
+:: Note: If under running under cygwin use "/cygdrive/c/..." for "C:/..."
+:: Example:
+::
+::     Set the CLASSPATH to a jar file and a directory.  Note that
+::     "classes dir" is a directory of class files with a space in the name.
+::
+:: CLASSPATH="usr/local/Product1/lib/product.jar"
+:: CLASSPATH="${CLASSPATH}:../MyProject/classes dir"
+::
+SET CLASSPATH="@classpath@"
+
+:: JVM parameters
+:: If you want to modify the default parameters (e.g. maximum heap size -Xmx)
+:: for the Java virtual machine set the local variable JVM_PARAMETERS below
+:: Example:
+:: JVM_PARAMETERS=-Xms100M -Xmx200M
+::
+:: Below are the JVM parameters needed to do remote debugging using Intellij
+:: IDEA.  Uncomment and then do: JVM_PARAMETERS="$IDEA_REMOTE_DEBUG_PARAMS"
+:: IDEA_REMOTE_DEBUG_PARAMS="-Xdebug -Xnoagent -Djava.compiler=NONE -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5005"
+::
+:: JVM_PARAMETERS=
+
+:: ---------------------------------------------------------------------------
+:: Default configuration. Do not modify below this line.
+:: ---------------------------------------------------------------------------
+:: Application specific parameters
+
+SET MAIN_CLASS=@main.class@
+SET JVM_PARAMS=@jvm.params@
+
+:: Try to find java virtual machine
+IF NOT DEFINED JAVA (
+  IF NOT DEFINED JAVA_HOME SET JAVA="java.exe"
+  IF DEFINED JAVA_HOME SET JAVA="%JAVA_HOME%\bin\java.exe"
+)
+
+:: Run program
+%JAVA% %JVM_PARAMS% %JVM_PARAMETERS% -classpath %CLASSPATH% %MAIN_CLASS% %*
diff --git a/hyracks-core/src/main/scripts/run.sh b/hyracks-core/src/main/scripts/run.sh
new file mode 100644
index 0000000..df04b73
--- /dev/null
+++ b/hyracks-core/src/main/scripts/run.sh
@@ -0,0 +1,100 @@
+#!/bin/sh
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# JAVA classpath
+# Use the local variable CLASSPATH to add custom entries (e.g. JDBC drivers) to
+# the classpath. Separate multiple paths with ":". Enclose the value
+# in double quotes. Adding additional files or locations on separate
+# lines makes things clearer.
+# Note: If under running under cygwin use "/cygdrive/c/..." for "C:/..."
+# Example:
+#
+#     Set the CLASSPATH to a jar file and a directory.  Note that
+#     "classes dir" is a directory of class files with a space in the name.
+#
+# CLASSPATH="usr/local/Product1/lib/product.jar"
+# CLASSPATH="${CLASSPATH}:../MyProject/classes dir"
+#
+if [ -z "$CLASSPATH" ]; then
+    CLASSPATH="@classpath@"
+else
+    CLASSPATH="@classpath@:$CLASSPATH"
+fi
+
+# JVM parameters
+# If you want to modify the default parameters (e.g. maximum heap size -Xmx)
+# for the Java virtual machine set the local variable JVM_PARAMETERS below
+# Example:
+# JVM_PARAMETERS=-Xms100M -Xmx200M
+#
+# Below are the JVM parameters needed to do remote debugging using Intellij
+# IDEA.  Uncomment and then do: JVM_PARAMETERS="$IDEA_REMOTE_DEBUG_PARAMS"
+# IDEA_REMOTE_DEBUG_PARAMS="-Xdebug -Xnoagent -Djava.compiler=NONE -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5005"
+#
+# JVM_PARAMETERS=
+
+#run with shared memory setup
+#if [ -n "${RUN_SHARED_MEM}"]; then
+#  JVM_PARAMETERS="${JVM_PARAMETERS} -Xdebug -Xnoagent -Djava.compiler=NONE -Xrunjdwp:transport=dt_shmem,server=n,address=javadebug,suspend=y"
+#fi
+
+# ---------------------------------------------------------------------------
+# Default configuration. Do not modify below this line.
+# ---------------------------------------------------------------------------
+# Application specific parameters
+
+MAIN_CLASS="@main.class@"
+JVM_PARAMS="@jvm.params@"
+
+# Cygwin support.  $cygwin _must_ be set to either true or false.
+case "`uname`" in
+  CYGWIN*) cygwin=true ;;
+  *) cygwin=false ;;
+esac
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched
+if $cygwin; then
+  [ -n "$JAVA_HOME" ] &&
+    JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+  [ -n "$CLASSPATH" ] &&
+    CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
+fi
+
+# Try to find java virtual machine
+if [ -z "${JAVA}" ];  then
+  if [ -z "${JAVA_HOME}" ]; then
+    JAVA=java
+  else
+    JAVA=${JAVA_HOME}/bin/java
+  fi
+fi
+
+# Try to find directory where this script is located
+COMMAND="${PWD}/$0"
+if [ ! -f "${COMMAND}" ]; then
+	COMMAND="$0"
+fi
+BASEDIR=`expr "${COMMAND}" : '\(.*\)/\.*'`
+
+# For Cygwin, switch paths to Windows format before running java
+if $cygwin; then
+#  JAVA=`cygpath --path --windows "$JAVA"`
+  CLASSPATH=`cygpath --path --windows "$CLASSPATH"`
+fi
+
+# Run program
+${JAVA} ${JVM_PARAMS} ${JVM_PARAMETERS} -classpath "${CLASSPATH}" ${MAIN_CLASS} $*
diff --git a/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java b/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
new file mode 100644
index 0000000..1467de5
--- /dev/null
+++ b/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
@@ -0,0 +1,143 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.tests.comm;
+
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+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.comm.io.FrameDeserializingDataReader;
+import edu.uci.ics.hyracks.comm.io.SerializingDataWriter;
+import edu.uci.ics.hyracks.context.HyracksContext;
+import edu.uci.ics.hyracks.coreops.data.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.coreops.data.StringSerializerDeserializer;
+
+public class SerializationDeserializationTest {
+    private static final String DBLP_FILE = "data/dblp.txt";
+
+    private static class SerDeserRunner {
+        private final HyracksContext ctx;
+        private static final int FRAME_SIZE = 32768;
+        private RecordDescriptor rDes;
+        private List<ByteBuffer> buffers;
+
+        public SerDeserRunner(RecordDescriptor rDes) {
+            ctx = new HyracksContext(FRAME_SIZE);
+            this.rDes = rDes;
+            buffers = new ArrayList<ByteBuffer>();
+        }
+
+        public IOpenableDataWriter<Object[]> createWriter() {
+            return new SerializingDataWriter(ctx, rDes, new IFrameWriter() {
+                @Override
+                public void open() throws HyracksDataException {
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    ByteBuffer toBuf = ctx.getResourceManager().allocateFrame();
+                    toBuf.put(buffer);
+                    buffers.add(toBuf);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+
+                }
+            });
+        }
+
+        public IOpenableDataReader<Object[]> createDataReader() {
+            return new FrameDeserializingDataReader(ctx, new IFrameReader() {
+                private int i;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    i = 0;
+                }
+
+                @Override
+                public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    if (i < buffers.size()) {
+                        ByteBuffer buf = buffers.get(i);
+                        buf.flip();
+                        buffer.put(buf);
+                        buffer.flip();
+                        ++i;
+                        return true;
+                    }
+                    return false;
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+
+                }
+            }, rDes);
+        }
+    }
+
+    private interface LineProcessor {
+        public void process(String line, IDataWriter<Object[]> writer) throws Exception;
+    }
+
+    private void run(RecordDescriptor rDes, LineProcessor lp) throws Exception {
+        SerDeserRunner runner = new SerDeserRunner(rDes);
+        IOpenableDataWriter<Object[]> writer = runner.createWriter();
+        writer.open();
+        BufferedReader in = new BufferedReader(new FileReader(DBLP_FILE));
+        String line;
+        while ((line = in.readLine()) != null) {
+            lp.process(line, writer);
+        }
+        writer.close();
+
+        IOpenableDataReader<Object[]> reader = runner.createDataReader();
+        reader.open();
+        Object[] arr;
+        while ((arr = reader.readData()) != null) {
+            System.err.println(arr[0] + " " + arr[1]);
+        }
+        reader.close();
+    }
+
+    @Test
+    public void serdeser01() throws Exception {
+        RecordDescriptor rDes = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+        LineProcessor processor = new LineProcessor() {
+            @Override
+            public void process(String line, IDataWriter<Object[]> writer) throws Exception {
+                String[] splits = line.split(" ");
+                for (String s : splits) {
+                    writer.writeData(new Object[] { s, Integer.valueOf(1) });
+                }
+            }
+        };
+        run(rDes, processor);
+    }
+}
diff --git a/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
new file mode 100644
index 0000000..94bc57a
--- /dev/null
+++ b/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.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.integration;
+
+import java.util.EnumSet;
+import java.util.UUID;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.config.CCConfig;
+import edu.uci.ics.hyracks.config.NCConfig;
+import edu.uci.ics.hyracks.controller.ClusterControllerService;
+import edu.uci.ics.hyracks.controller.NodeControllerService;
+
+public abstract class AbstractIntegrationTest {
+    public static final String NC1_ID = "nc1";
+    public static final String NC2_ID = "nc2";
+
+    protected static ClusterControllerService cc;
+    protected static NodeControllerService nc1;
+    protected static NodeControllerService nc2;
+
+    @BeforeClass
+    public static void init() throws Exception {
+        CCConfig ccConfig = new CCConfig();
+        ccConfig.port = 39001;
+        cc = new ClusterControllerService(ccConfig);
+        cc.start();
+
+        NCConfig ncConfig1 = new NCConfig();
+        ncConfig1.ccHost = "localhost";
+        ncConfig1.ccPort = 39001;
+        ncConfig1.dataIPAddress = "127.0.0.1";
+        ncConfig1.nodeId = NC1_ID;
+        nc1 = new NodeControllerService(ncConfig1);
+        nc1.start();
+
+        NCConfig ncConfig2 = new NCConfig();
+        ncConfig2.ccHost = "localhost";
+        ncConfig2.ccPort = 39001;
+        ncConfig2.dataIPAddress = "127.0.0.1";
+        ncConfig2.nodeId = NC2_ID;
+        nc2 = new NodeControllerService(ncConfig2);
+        nc2.start();
+    }
+
+    @AfterClass
+    public static void deinit() throws Exception {
+        nc2.stop();
+        nc1.stop();
+        cc.stop();
+    }
+
+    void runTest(JobSpecification spec) throws Exception {
+        UUID jobId = cc.createJob(spec, EnumSet.of(JobFlag.COLLECT_FRAME_COUNTS));
+        System.err.println(spec.toJSON());
+        cc.start(jobId);
+        System.err.print(jobId);
+        System.err.println(cc.waitForCompletion(jobId));
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java b/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
new file mode 100644
index 0000000..a3beb11
--- /dev/null
+++ b/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
@@ -0,0 +1,252 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.tests.integration;
+
+import java.io.File;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparatorFactory;
+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.JobSpecification;
+import edu.uci.ics.hyracks.coreops.ExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.coreops.InMemorySortOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.coreops.MToNReplicatingConnectorDescriptor;
+import edu.uci.ics.hyracks.coreops.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.coreops.PrinterOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.aggregators.SumStringGroupAggregator;
+import edu.uci.ics.hyracks.coreops.data.StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.coreops.data.StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.coreops.data.StringComparatorFactory;
+import edu.uci.ics.hyracks.coreops.data.StringSerializerDeserializer;
+import edu.uci.ics.hyracks.coreops.file.CSVFileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.file.FileSplit;
+import edu.uci.ics.hyracks.coreops.group.PreclusteredGroupOperatorDescriptor;
+
+public class CountOfCountsTest extends AbstractIntegrationTest {
+    @Test
+    public void countOfCountsSingleNC() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] splits = new FileSplit[] { new FileSplit(NC1_ID, new File("data/words.txt")) };
+        RecordDescriptor desc = new RecordDescriptor(
+                new ISerializerDeserializer[] { StringSerializerDeserializer.INSTANCE });
+
+        CSVFileScanOperatorDescriptor csvScanner = new CSVFileScanOperatorDescriptor(spec, splits, desc);
+        PartitionConstraint csvPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        csvScanner.setPartitionConstraint(csvPartitionConstraint);
+
+        InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 0 },
+                new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE }, desc);
+        PartitionConstraint sorterPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        sorter.setPartitionConstraint(sorterPartitionConstraint);
+
+        RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE });
+        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
+                new IComparatorFactory[] { StringComparatorFactory.INSTANCE }, new SumStringGroupAggregator(0), desc2);
+        PartitionConstraint groupPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        group.setPartitionConstraint(groupPartitionConstraint);
+
+        InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
+                new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE }, desc2);
+        PartitionConstraint sorterPartitionConstraint2 = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        sorter2.setPartitionConstraint(sorterPartitionConstraint2);
+
+        PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
+                new IComparatorFactory[] { StringComparatorFactory.INSTANCE }, new SumStringGroupAggregator(1), desc2);
+        PartitionConstraint groupPartitionConstraint2 = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        group2.setPartitionConstraint(groupPartitionConstraint2);
+
+        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
+        PartitionConstraint printerPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        printer.setPartitionConstraint(printerPartitionConstraint);
+
+        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 0 },
+                        new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+        spec.connect(conn1, csvScanner, 0, sorter, 0);
+
+        IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
+        spec.connect(conn2, sorter, 0, group, 0);
+
+        IConnectorDescriptor conn3 = new MToNHashPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 1 },
+                        new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+        spec.connect(conn3, group, 0, sorter2, 0);
+
+        IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
+        spec.connect(conn4, sorter2, 0, group2, 0);
+
+        IConnectorDescriptor conn5 = new MToNReplicatingConnectorDescriptor(spec);
+        spec.connect(conn5, group2, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Test
+    public void countOfCountsMultiNC() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] splits = new FileSplit[] { new FileSplit(NC1_ID, new File("data/words.txt")) };
+        RecordDescriptor desc = new RecordDescriptor(
+                new ISerializerDeserializer[] { StringSerializerDeserializer.INSTANCE });
+
+        CSVFileScanOperatorDescriptor csvScanner = new CSVFileScanOperatorDescriptor(spec, splits, desc);
+        PartitionConstraint csvPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        csvScanner.setPartitionConstraint(csvPartitionConstraint);
+
+        InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 0 },
+                new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE }, desc);
+        PartitionConstraint sorterPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID),
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        sorter.setPartitionConstraint(sorterPartitionConstraint);
+
+        RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE });
+        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
+                new IComparatorFactory[] { StringComparatorFactory.INSTANCE }, new SumStringGroupAggregator(0), desc2);
+        PartitionConstraint groupPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID),
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        group.setPartitionConstraint(groupPartitionConstraint);
+
+        InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
+                new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE }, desc2);
+        PartitionConstraint sorterPartitionConstraint2 = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        sorter2.setPartitionConstraint(sorterPartitionConstraint2);
+
+        PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
+                new IComparatorFactory[] { StringComparatorFactory.INSTANCE }, new SumStringGroupAggregator(1), desc2);
+        PartitionConstraint groupPartitionConstraint2 = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        group2.setPartitionConstraint(groupPartitionConstraint2);
+
+        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
+        PartitionConstraint printerPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        printer.setPartitionConstraint(printerPartitionConstraint);
+
+        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 0 },
+                        new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+        spec.connect(conn1, csvScanner, 0, sorter, 0);
+
+        IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
+        spec.connect(conn2, sorter, 0, group, 0);
+
+        IConnectorDescriptor conn3 = new MToNHashPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 1 },
+                        new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+        spec.connect(conn3, group, 0, sorter2, 0);
+
+        IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
+        spec.connect(conn4, sorter2, 0, group2, 0);
+
+        IConnectorDescriptor conn5 = new MToNReplicatingConnectorDescriptor(spec);
+        spec.connect(conn5, group2, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Test
+    public void countOfCountsExternalSortMultiNC() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] splits = new FileSplit[] { new FileSplit(NC1_ID, new File("data/words.txt")) };
+        RecordDescriptor desc = new RecordDescriptor(
+                new ISerializerDeserializer[] { StringSerializerDeserializer.INSTANCE });
+
+        CSVFileScanOperatorDescriptor csvScanner = new CSVFileScanOperatorDescriptor(spec, splits, desc);
+        PartitionConstraint csvPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        csvScanner.setPartitionConstraint(csvPartitionConstraint);
+
+        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 3, new int[] { 0 },
+                new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE }, desc);
+        PartitionConstraint sorterPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID),
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        sorter.setPartitionConstraint(sorterPartitionConstraint);
+
+        RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE });
+        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
+                new IComparatorFactory[] { StringComparatorFactory.INSTANCE }, new SumStringGroupAggregator(0), desc2);
+        PartitionConstraint groupPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID),
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        group.setPartitionConstraint(groupPartitionConstraint);
+
+        ExternalSortOperatorDescriptor sorter2 = new ExternalSortOperatorDescriptor(spec, 3, new int[] { 1 },
+                new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE }, desc2);
+        PartitionConstraint sorterPartitionConstraint2 = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        sorter2.setPartitionConstraint(sorterPartitionConstraint2);
+
+        PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
+                new IComparatorFactory[] { StringComparatorFactory.INSTANCE }, new SumStringGroupAggregator(1), desc2);
+        PartitionConstraint groupPartitionConstraint2 = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        group2.setPartitionConstraint(groupPartitionConstraint2);
+
+        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
+        PartitionConstraint printerPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        printer.setPartitionConstraint(printerPartitionConstraint);
+
+        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 0 },
+                        new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+        spec.connect(conn1, csvScanner, 0, sorter, 0);
+
+        IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
+        spec.connect(conn2, sorter, 0, group, 0);
+
+        IConnectorDescriptor conn3 = new MToNHashPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 1 },
+                        new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+        spec.connect(conn3, group, 0, sorter2, 0);
+
+        IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
+        spec.connect(conn4, sorter2, 0, group2, 0);
+
+        IConnectorDescriptor conn5 = new MToNReplicatingConnectorDescriptor(spec);
+        spec.connect(conn5, group2, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java b/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
new file mode 100644
index 0000000..d619bba
--- /dev/null
+++ b/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.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.tests.integration;
+
+import java.io.File;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+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.JobSpecification;
+import edu.uci.ics.hyracks.coreops.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.coreops.PrinterOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.data.StringSerializerDeserializer;
+import edu.uci.ics.hyracks.coreops.file.CSVFileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.file.FileSplit;
+
+public class ScanPrintTest extends AbstractIntegrationTest {
+    @Test
+    public void scanPrint01() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] splits = new FileSplit[] { new FileSplit(NC2_ID, new File("data/words.txt")),
+                new FileSplit(NC1_ID, new File("data/words.txt")) };
+        RecordDescriptor desc = new RecordDescriptor(
+                new ISerializerDeserializer[] { StringSerializerDeserializer.INSTANCE });
+
+        CSVFileScanOperatorDescriptor csvScanner = new CSVFileScanOperatorDescriptor(spec, splits, desc);
+        PartitionConstraint csvPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC2_ID), new AbsoluteLocationConstraint(NC1_ID) });
+        csvScanner.setPartitionConstraint(csvPartitionConstraint);
+
+        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
+        PartitionConstraint printerPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC2_ID), new AbsoluteLocationConstraint(NC1_ID) });
+        printer.setPartitionConstraint(printerPartitionConstraint);
+
+        IConnectorDescriptor conn = new OneToOneConnectorDescriptor(spec);
+        spec.connect(conn, csvScanner, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
new file mode 100644
index 0000000..cd254e2
--- /dev/null
+++ b/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.tests.integration;
+
+import java.io.File;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+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.JobSpecification;
+import edu.uci.ics.hyracks.coreops.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.coreops.InMemorySortOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.MToNHashPartitioningMergingConnectorDescriptor;
+import edu.uci.ics.hyracks.coreops.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.coreops.PrinterOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.data.StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.coreops.data.StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.coreops.data.StringSerializerDeserializer;
+import edu.uci.ics.hyracks.coreops.file.CSVFileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.file.FileSplit;
+
+public class SortMergeTest extends AbstractIntegrationTest {
+    @Test
+    public void sortMergeTest01() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] ordersSplits = new FileSplit[] {
+                new FileSplit(NC1_ID, new File("data/tpch0.001/orders-part1.tbl")),
+                new FileSplit(NC2_ID, new File("data/tpch0.001/orders-part2.tbl")) };
+        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE });
+
+        CSVFileScanOperatorDescriptor ordScanner = new CSVFileScanOperatorDescriptor(spec, ordersSplits, ordersDesc,
+                '|', "'\"");
+        PartitionConstraint ordersPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+
+        InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
+                new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+        PartitionConstraint sortersPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        sorter.setPartitionConstraint(sortersPartitionConstraint);
+
+        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
+        PartitionConstraint printerPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        printer.setPartitionConstraint(printerPartitionConstraint);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
+
+        spec.connect(new MToNHashPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
+                new int[] { 1 }, new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }),
+                new int[] { 1 }, new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE }), sorter, 0,
+                printer, 0);
+        
+        runTest(spec);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
new file mode 100644
index 0000000..593eac1
--- /dev/null
+++ b/hyracks-core/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -0,0 +1,300 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.tests.integration;
+
+import java.io.File;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.coreops.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.coreops.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.coreops.MToNReplicatingConnectorDescriptor;
+import edu.uci.ics.hyracks.coreops.MaterializingOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.coreops.PrinterOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.data.StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.coreops.data.StringBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.coreops.data.StringSerializerDeserializer;
+import edu.uci.ics.hyracks.coreops.file.CSVFileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.coreops.file.FileSplit;
+import edu.uci.ics.hyracks.coreops.join.InMemoryHashJoinOperatorDescriptor;
+
+public class TPCHCustomerOrderHashJoinTest extends AbstractIntegrationTest {
+    /*
+     * TPCH Customer table:
+     * CREATE TABLE CUSTOMER (
+     * C_CUSTKEY INTEGER NOT NULL,
+     * C_NAME VARCHAR(25) NOT NULL,
+     * C_ADDRESS VARCHAR(40) NOT NULL,
+     * C_NATIONKEY INTEGER NOT NULL,
+     * C_PHONE CHAR(15) NOT NULL,
+     * C_ACCTBAL DECIMAL(15,2) NOT NULL,
+     * C_MKTSEGMENT CHAR(10) NOT NULL,
+     * C_COMMENT VARCHAR(117) NOT NULL
+     * );
+     * TPCH Orders table:
+     * CREATE TABLE ORDERS (
+     * O_ORDERKEY INTEGER NOT NULL,
+     * O_CUSTKEY INTEGER NOT NULL,
+     * O_ORDERSTATUS CHAR(1) NOT NULL,
+     * O_TOTALPRICE DECIMAL(15,2) NOT NULL,
+     * O_ORDERDATE DATE NOT NULL,
+     * O_ORDERPRIORITY CHAR(15) NOT NULL,
+     * O_CLERK CHAR(15) NOT NULL,
+     * O_SHIPPRIORITY INTEGER NOT NULL,
+     * O_COMMENT VARCHAR(79) NOT NULL
+     * );
+     */
+
+    @Test
+    public void customerOrderCIDJoin() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] custSplits = new FileSplit[] { new FileSplit(NC1_ID, new File("data/tpch0.001/customer.tbl")) };
+        RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE });
+
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new File("data/tpch0.001/orders.tbl")) };
+        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE });
+
+        RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE });
+
+        CSVFileScanOperatorDescriptor ordScanner = new CSVFileScanOperatorDescriptor(spec, ordersSplits, ordersDesc,
+                '|', "'\"");
+        PartitionConstraint ordersPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+
+        CSVFileScanOperatorDescriptor custScanner = new CSVFileScanOperatorDescriptor(spec, custSplits, custDesc, '|',
+                "'\"");
+        PartitionConstraint custPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        custScanner.setPartitionConstraint(custPartitionConstraint);
+
+        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
+                new int[] { 0 }, new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE },
+                new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
+        PartitionConstraint joinPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        join.setPartitionConstraint(joinPartitionConstraint);
+
+        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
+        PartitionConstraint printerPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        printer.setPartitionConstraint(printerPartitionConstraint);
+
+        IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
+        spec.connect(ordJoinConn, ordScanner, 0, join, 0);
+
+        IConnectorDescriptor custJoinConn = new OneToOneConnectorDescriptor(spec);
+        spec.connect(custJoinConn, custScanner, 0, join, 1);
+
+        IConnectorDescriptor joinPrinterConn = new OneToOneConnectorDescriptor(spec);
+        spec.connect(joinPrinterConn, join, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Test
+    public void customerOrderCIDJoinMulti() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] custSplits = new FileSplit[] {
+                new FileSplit(NC1_ID, new File("data/tpch0.001/customer-part1.tbl")),
+                new FileSplit(NC2_ID, new File("data/tpch0.001/customer-part2.tbl")) };
+        RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE });
+
+        FileSplit[] ordersSplits = new FileSplit[] {
+                new FileSplit(NC1_ID, new File("data/tpch0.001/orders-part1.tbl")),
+                new FileSplit(NC2_ID, new File("data/tpch0.001/orders-part2.tbl")) };
+        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE });
+
+        RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE });
+
+        CSVFileScanOperatorDescriptor ordScanner = new CSVFileScanOperatorDescriptor(spec, ordersSplits, ordersDesc,
+                '|', "'\"");
+        PartitionConstraint ordersPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+
+        CSVFileScanOperatorDescriptor custScanner = new CSVFileScanOperatorDescriptor(spec, custSplits, custDesc, '|',
+                "'\"");
+        PartitionConstraint custPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        custScanner.setPartitionConstraint(custPartitionConstraint);
+
+        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
+                new int[] { 0 }, new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE },
+                new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
+        PartitionConstraint joinPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        join.setPartitionConstraint(joinPartitionConstraint);
+
+        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
+        PartitionConstraint printerPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        printer.setPartitionConstraint(printerPartitionConstraint);
+
+        IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 1 },
+                        new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+        spec.connect(ordJoinConn, ordScanner, 0, join, 0);
+
+        IConnectorDescriptor custJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 0 },
+                        new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+        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 customerOrderCIDJoinMultiMaterialized() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] custSplits = new FileSplit[] {
+                new FileSplit(NC1_ID, new File("data/tpch0.001/customer-part1.tbl")),
+                new FileSplit(NC2_ID, new File("data/tpch0.001/customer-part2.tbl")) };
+        RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE });
+
+        FileSplit[] ordersSplits = new FileSplit[] {
+                new FileSplit(NC1_ID, new File("data/tpch0.001/orders-part1.tbl")),
+                new FileSplit(NC2_ID, new File("data/tpch0.001/orders-part2.tbl")) };
+        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE });
+
+        RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE, StringSerializerDeserializer.INSTANCE,
+                StringSerializerDeserializer.INSTANCE });
+
+        CSVFileScanOperatorDescriptor ordScanner = new CSVFileScanOperatorDescriptor(spec, ordersSplits, ordersDesc,
+                '|', "'\"");
+        PartitionConstraint ordersPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+
+        CSVFileScanOperatorDescriptor custScanner = new CSVFileScanOperatorDescriptor(spec, custSplits, custDesc, '|',
+                "'\"");
+        PartitionConstraint custPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        custScanner.setPartitionConstraint(custPartitionConstraint);
+
+        MaterializingOperatorDescriptor ordMat = new MaterializingOperatorDescriptor(spec, ordersDesc);
+        ordMat.setPartitionConstraint(new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) }));
+
+        MaterializingOperatorDescriptor custMat = new MaterializingOperatorDescriptor(spec, custDesc);
+        custMat.setPartitionConstraint(new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) }));
+
+        InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
+                new int[] { 0 }, new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE },
+                new IBinaryComparatorFactory[] { StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
+        PartitionConstraint joinPartitionConstraint = new PartitionConstraint(new LocationConstraint[] {
+                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
+        join.setPartitionConstraint(joinPartitionConstraint);
+
+        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
+        PartitionConstraint printerPartitionConstraint = new PartitionConstraint(
+                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
+        printer.setPartitionConstraint(printerPartitionConstraint);
+
+        IConnectorDescriptor ordPartConn = new MToNHashPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 1 },
+                        new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+        spec.connect(ordPartConn, ordScanner, 0, ordMat, 0);
+
+        IConnectorDescriptor custPartConn = new MToNHashPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 0 },
+                        new IBinaryHashFunctionFactory[] { StringBinaryHashFunctionFactory.INSTANCE }));
+        spec.connect(custPartConn, custScanner, 0, custMat, 0);
+
+        IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
+        spec.connect(ordJoinConn, ordMat, 0, join, 0);
+
+        IConnectorDescriptor custJoinConn = new OneToOneConnectorDescriptor(spec);
+        spec.connect(custJoinConn, custMat, 0, join, 1);
+
+        IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
+        spec.connect(joinPrinterConn, join, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+}
\ No newline at end of file