Merged hyracks_scheduling branch into hyracks_dev_next

git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_dev_next@543 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
new file mode 100644
index 0000000..e621a09
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.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.channels;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+
+public interface IInputChannel {
+    public void registerMonitor(IInputChannelMonitor monitor) throws HyracksException;
+
+    public void setAttachment(Object attachment);
+
+    public Object getAttachment();
+
+    public ByteBuffer getNextBuffer();
+
+    public void recycleBuffer(ByteBuffer buffer);
+
+    public void open() throws HyracksDataException;
+
+    public void close() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannelMonitor.java
similarity index 74%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannelMonitor.java
index c884998..dd91f28 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannelMonitor.java
@@ -12,9 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.channels;
 
-public enum Direction {
-    INPUT,
-    OUTPUT,
+public interface IInputChannelMonitor {
+    public void notifyDataAvailability(IInputChannel channel, int nFrames);
+
+    public void notifyEndOfStream(IInputChannel channel);
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/AbstractHyracksConnection.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/AbstractHyracksConnection.java
index 779e9ac..1377e37 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/AbstractHyracksConnection.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/AbstractHyracksConnection.java
@@ -16,7 +16,6 @@
 
 import java.io.File;
 import java.util.EnumSet;
-import java.util.UUID;
 
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
@@ -26,6 +25,7 @@
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
@@ -65,27 +65,27 @@
     }
 
     @Override
-    public UUID createJob(String appName, JobSpecification jobSpec) throws Exception {
+    public JobId createJob(String appName, JobSpecification jobSpec) throws Exception {
         return createJob(appName, jobSpec, EnumSet.noneOf(JobFlag.class));
     }
 
     @Override
-    public UUID createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+    public JobId createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
         return hci.createJob(appName, JavaSerializationUtils.serialize(jobSpec), jobFlags);
     }
 
     @Override
-    public JobStatus getJobStatus(UUID jobId) throws Exception {
+    public JobStatus getJobStatus(JobId jobId) throws Exception {
         return hci.getJobStatus(jobId);
     }
 
     @Override
-    public void start(UUID jobId) throws Exception {
+    public void start(JobId jobId) throws Exception {
         hci.start(jobId);
     }
 
     @Override
-    public void waitForCompletion(UUID jobId) throws Exception {
+    public void waitForCompletion(JobId jobId) throws Exception {
         hci.waitForCompletion(jobId);
     }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
index a164fee..659a181 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
@@ -16,9 +16,9 @@
 
 import java.io.File;
 import java.util.EnumSet;
-import java.util.UUID;
 
 import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 
@@ -27,13 +27,13 @@
 
     public void destroyApplication(String appName) throws Exception;
 
-    public UUID createJob(String appName, JobSpecification jobSpec) throws Exception;
+    public JobId createJob(String appName, JobSpecification jobSpec) throws Exception;
 
-    public UUID createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
+    public JobId createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
 
-    public JobStatus getJobStatus(UUID jobId) throws Exception;
+    public JobStatus getJobStatus(JobId jobId) throws Exception;
 
-    public void start(UUID jobId) throws Exception;
+    public void start(JobId jobId) throws Exception;
 
-    public void waitForCompletion(UUID jobId) throws Exception;
+    public void waitForCompletion(JobId jobId) throws Exception;
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
index c4bf0e6..f57ac97 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
@@ -16,9 +16,9 @@
 
 import java.rmi.Remote;
 import java.util.EnumSet;
-import java.util.UUID;
 
 import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 
 public interface IHyracksClientInterface extends Remote {
@@ -30,11 +30,11 @@
 
     public void destroyApplication(String appName) throws Exception;
 
-    public UUID createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
+    public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
 
-    public JobStatus getJobStatus(UUID jobId) throws Exception;
+    public JobStatus getJobStatus(JobId jobId) throws Exception;
 
-    public void start(UUID jobId) throws Exception;
+    public void start(JobId jobId) throws Exception;
 
-    public void waitForCompletion(UUID jobId) throws Exception;
+    public void waitForCompletion(JobId jobId) throws Exception;
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionEntry.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionEntry.java
index 9e3cbbe..ec05d0b 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionEntry.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionEntry.java
@@ -24,8 +24,6 @@
 
     SelectionKey getSelectionKey();
 
-    void setDataReceiveListener(IDataReceiveListener listener);
-
     void attach(Object attachment);
 
     Object getAttachment();
@@ -38,11 +36,5 @@
 
     UUID getStageId();
 
-    void setJobId(UUID jobId);
-
-    void setStageId(UUID stageId);
-
     boolean aborted();
-
-    void abort();
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionCollector.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionCollector.java
new file mode 100644
index 0000000..89e6da3
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionCollector.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.comm;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public interface IPartitionCollector {
+    public JobId getJobId();
+
+    public ConnectorDescriptorId getConnectorId();
+
+    public int getReceiverIndex();
+
+    public void open() throws HyracksException;
+
+    public void addPartitions(Collection<PartitionChannel> partitions) throws HyracksException;
+
+    public IFrameReader getReader() throws HyracksException;
+
+    public void close() throws HyracksException;
+
+    public Collection<PartitionId> getRequiredPartitionIds() throws HyracksException;
+
+    public void abort();
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionWriterFactory.java
similarity index 77%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionWriterFactory.java
index 463e4e1..e15ec6d 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionWriterFactory.java
@@ -14,10 +14,8 @@
  */
 package edu.uci.ics.hyracks.api.comm;
 
-import java.io.IOException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
-public interface IDataReceiveListener {
-    public void dataReceived(IConnectionEntry entry) throws IOException;
-
-    public void eos(IConnectionEntry entry);
+public interface IPartitionWriterFactory {
+    public IFrameWriter createFrameWriter(int receiverIndex) throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListenerFactory.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IReceiver.java
similarity index 65%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListenerFactory.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IReceiver.java
index 7e74aee..8ae10c4 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListenerFactory.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IReceiver.java
@@ -14,12 +14,14 @@
  */
 package edu.uci.ics.hyracks.api.comm;
 
-import java.util.UUID;
+import java.util.BitSet;
 
-public interface IDataReceiveListenerFactory {
-    public IDataReceiveListener getDataReceiveListener(UUID endpointUUID, IConnectionEntry entry, int senderIndex);
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 
-    public UUID getJobId();
+public interface IReceiver {
+    public void setReceiverEventListener(IReceiverEventListener listener) throws HyracksException;
 
-    public UUID getStageId();
+    public void init(int sender) throws HyracksException;
+
+    public ISenderProxy getReadySender(BitSet senderMask, int senderIndexPreference);
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IReceiverEventListener.java
similarity index 78%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IReceiverEventListener.java
index 463e4e1..98c083e 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IReceiverEventListener.java
@@ -14,10 +14,8 @@
  */
 package edu.uci.ics.hyracks.api.comm;
 
-import java.io.IOException;
+public interface IReceiverEventListener {
+    public void notifySenderAvailability(int sender);
 
-public interface IDataReceiveListener {
-    public void dataReceived(IConnectionEntry entry) throws IOException;
-
-    public void eos(IConnectionEntry entry);
+    public void notifySenderCount(int senderCount);
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/ISender.java
similarity index 82%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/ISender.java
index c884998..88c8f27 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/ISender.java
@@ -12,9 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.comm;
 
-public enum Direction {
-    INPUT,
-    OUTPUT,
+public interface ISender {
+    public IFrameWriter createSenderWriter(int receiverIndex);
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/ISenderProxy.java
similarity index 78%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/ISenderProxy.java
index c884998..6b40bae 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/ISenderProxy.java
@@ -12,9 +12,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.comm;
 
-public enum Direction {
-    INPUT,
-    OUTPUT,
+import java.nio.ByteBuffer;
+
+public interface ISenderProxy {
+    public ByteBuffer getReadBuffer();
+
+    public int getSenderIndex();
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/PartitionChannel.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/PartitionChannel.java
new file mode 100644
index 0000000..b75ed07
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/PartitionChannel.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.comm;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class PartitionChannel {
+    private PartitionId partitionId;
+
+    private IInputChannel channel;
+
+    public PartitionChannel(PartitionId partitionId, IInputChannel channel) {
+        this.partitionId = partitionId;
+        this.channel = channel;
+    }
+
+    public PartitionId getPartitionId() {
+        return partitionId;
+    }
+
+    public IInputChannel getInputChannel() {
+        return channel;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/Constraint.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/Constraint.java
new file mode 100644
index 0000000..0668e00
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/Constraint.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.api.constraints;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+
+public class Constraint implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final LValueConstraintExpression lValue;
+
+    private final ConstraintExpression rValue;
+
+    public Constraint(LValueConstraintExpression lValue, ConstraintExpression rValue) {
+        this.lValue = lValue;
+        this.rValue = rValue;
+    }
+
+    public LValueConstraintExpression getLValue() {
+        return lValue;
+    }
+
+    public ConstraintExpression getRValue() {
+        return rValue;
+    }
+
+    @Override
+    public String toString() {
+        return lValue + " in " + rValue;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintAcceptor.java
similarity index 81%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintAcceptor.java
index c884998..2e41b0a 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintAcceptor.java
@@ -12,9 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.constraints;
 
-public enum Direction {
-    INPUT,
-    OUTPUT,
+public interface IConstraintAcceptor {
+    public void addConstraint(Constraint constraint);
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintExpressionAcceptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintExpressionAcceptor.java
deleted file mode 100644
index 7e74814..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintExpressionAcceptor.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.constraints;
-
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
-
-public interface IConstraintExpressionAcceptor {
-    public void addConstraintExpression(ConstraintExpression constraintExpression);
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraintHelper.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraintHelper.java
index 309fca1..0e4ca69 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraintHelper.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraintHelper.java
@@ -14,45 +14,31 @@
  */
 package edu.uci.ics.hyracks.api.constraints;
 
-import java.util.ArrayList;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.constraints.expressions.BelongsToExpression;
 import edu.uci.ics.hyracks.api.constraints.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.EnumeratedCollectionExpression;
 import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
 import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.RelationalExpression;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 public class PartitionConstraintHelper {
     public static void addPartitionCountConstraint(JobSpecification spec, IOperatorDescriptor op, int count) {
-        spec.addUserConstraint(new RelationalExpression(new PartitionCountExpression(op.getOperatorId()),
-                new ConstantExpression(count), RelationalExpression.Operator.EQUAL));
+        spec.addUserConstraint(new Constraint(new PartitionCountExpression(op.getOperatorId()), new ConstantExpression(
+                count)));
     }
 
     public static void addLocationChoiceConstraint(JobSpecification spec, IOperatorDescriptor op, String[][] choices) {
         addPartitionCountConstraint(spec, op, choices.length);
         for (int i = 0; i < choices.length; ++i) {
-            String[] choice = choices[i];
-            List<ConstraintExpression> choiceExprs = new ArrayList<ConstraintExpression>();
-            for (String c : choice) {
-                choiceExprs.add(new ConstantExpression(c));
-            }
-            spec.addUserConstraint(new BelongsToExpression(new PartitionLocationExpression(op.getOperatorId(), i),
-                    new EnumeratedCollectionExpression(choiceExprs)));
+            spec.addUserConstraint(new Constraint(new PartitionLocationExpression(op.getOperatorId(), i),
+                    new ConstantExpression(choices[i])));
         }
     }
 
     public static void addAbsoluteLocationConstraint(JobSpecification spec, IOperatorDescriptor op, String... locations) {
         addPartitionCountConstraint(spec, op, locations.length);
         for (int i = 0; i < locations.length; ++i) {
-            List<ConstraintExpression> choiceExprs = new ArrayList<ConstraintExpression>();
-            choiceExprs.add(new ConstantExpression(locations[i]));
-            spec.addUserConstraint(new BelongsToExpression(new PartitionLocationExpression(op.getOperatorId(), i),
-                    new EnumeratedCollectionExpression(choiceExprs)));
+            spec.addUserConstraint(new Constraint(new PartitionLocationExpression(op.getOperatorId(), i),
+                    new ConstantExpression(locations[i])));
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/BelongsToExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/BelongsToExpression.java
deleted file mode 100644
index c307e3e..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/BelongsToExpression.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.constraints.expressions;
-
-import java.util.Collection;
-
-public final class BelongsToExpression extends ConstraintExpression {
-    private static final long serialVersionUID = 1L;
-
-    private final ConstraintExpression itemExpression;
-
-    private final ConstraintExpression setExpression;
-
-    public BelongsToExpression(ConstraintExpression itemExpression, ConstraintExpression setExpression) {
-        this.itemExpression = itemExpression;
-        this.setExpression = setExpression;
-    }
-
-    @Override
-    public ExpressionTag getTag() {
-        return ExpressionTag.BELONGS_TO;
-    }
-
-    public ConstraintExpression getItemExpression() {
-        return itemExpression;
-    }
-
-    public ConstraintExpression getSetExpression() {
-        return setExpression;
-    }
-
-    @Override
-    public void getChildren(Collection<ConstraintExpression> children) {
-        children.add(itemExpression);
-        children.add(setExpression);
-    }
-
-    @Override
-    protected void toString(StringBuilder buffer) {
-        buffer.append(getTag()).append('(');
-        itemExpression.toString(buffer);
-        buffer.append(", ");
-        setExpression.toString(buffer);
-        buffer.append(')');
-    }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ConstraintExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ConstraintExpression.java
index afd5051..811c048 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ConstraintExpression.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ConstraintExpression.java
@@ -23,11 +23,7 @@
     public enum ExpressionTag {
         CONSTANT,
         PARTITION_COUNT,
-        RESOURCE_SUM,
         PARTITION_LOCATION,
-        ENUMERATED_SET,
-        BELONGS_TO,
-        RELATIONAL,
     }
 
     public abstract ExpressionTag getTag();
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/EnumeratedCollectionExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/EnumeratedCollectionExpression.java
deleted file mode 100644
index d0ce706..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/EnumeratedCollectionExpression.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.constraints.expressions;
-
-import java.util.Collection;
-
-public class EnumeratedCollectionExpression extends ConstraintExpression {
-    private static final long serialVersionUID = 1L;
-
-    private final Collection<ConstraintExpression> enumeration;
-
-    public EnumeratedCollectionExpression(Collection<ConstraintExpression> enumeration) {
-        this.enumeration = enumeration;
-    }
-
-    @Override
-    public ExpressionTag getTag() {
-        return ExpressionTag.ENUMERATED_SET;
-    }
-
-    public Collection<ConstraintExpression> getMembers() {
-        return enumeration;
-    }
-
-    @Override
-    public void getChildren(Collection<ConstraintExpression> children) {
-        children.addAll(enumeration);
-    }
-
-    @Override
-    protected void toString(StringBuilder buffer) {
-        buffer.append(getTag()).append('(');
-        boolean first = true;
-        for (ConstraintExpression ce : enumeration) {
-            if (!first) {
-                buffer.append(", ");
-            }
-            first = false;
-            ce.toString(buffer);
-        }
-        buffer.append(')');
-    }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/LValueConstraintExpression.java
similarity index 76%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/LValueConstraintExpression.java
index c884998..7557460 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/LValueConstraintExpression.java
@@ -12,9 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.constraints.expressions;
 
-public enum Direction {
-    INPUT,
-    OUTPUT,
+public abstract class LValueConstraintExpression extends ConstraintExpression {
+    private static final long serialVersionUID = 1L;
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionCountExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionCountExpression.java
index c538ff7..dc73947 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionCountExpression.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionCountExpression.java
@@ -18,7 +18,7 @@
 
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
 
-public final class PartitionCountExpression extends ConstraintExpression {
+public final class PartitionCountExpression extends LValueConstraintExpression {
     private static final long serialVersionUID = 1L;
 
     private final OperatorDescriptorId opId;
@@ -44,4 +44,29 @@
     protected void toString(StringBuilder buffer) {
         buffer.append(getTag()).append('(').append(opId.toString()).append(')');
     }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((opId == null) ? 0 : opId.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        PartitionCountExpression other = (PartitionCountExpression) obj;
+        if (opId == null) {
+            if (other.opId != null)
+                return false;
+        } else if (!opId.equals(other.opId))
+            return false;
+        return true;
+    }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionLocationExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionLocationExpression.java
index 50e4e47..fbfdc1c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionLocationExpression.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionLocationExpression.java
@@ -18,7 +18,7 @@
 
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
 
-public final class PartitionLocationExpression extends ConstraintExpression {
+public final class PartitionLocationExpression extends LValueConstraintExpression {
     private static final long serialVersionUID = 1L;
 
     private final OperatorDescriptorId opId;
@@ -50,4 +50,32 @@
     protected void toString(StringBuilder buffer) {
         buffer.append(getTag()).append('(').append(opId.toString()).append(", ").append(partition).append(')');
     }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((opId == null) ? 0 : opId.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;
+        PartitionLocationExpression other = (PartitionLocationExpression) obj;
+        if (opId == null) {
+            if (other.opId != null)
+                return false;
+        } else if (!opId.equals(other.opId))
+            return false;
+        if (partition != other.partition)
+            return false;
+        return true;
+    }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/RelationalExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/RelationalExpression.java
deleted file mode 100644
index d90bce9..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/RelationalExpression.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.constraints.expressions;
-
-import java.util.Collection;
-
-public class RelationalExpression extends ConstraintExpression {
-    private static final long serialVersionUID = 1L;
-
-    public enum Operator {
-        EQUAL,
-        NOT_EQUAL,
-        LESS,
-        LESS_EQUAL,
-        GREATER,
-        GREATER_EQUAL
-    }
-
-    private final ConstraintExpression left;
-    private final ConstraintExpression right;
-    private final Operator op;
-
-    public RelationalExpression(ConstraintExpression left, ConstraintExpression right, Operator op) {
-        this.left = left;
-        this.right = right;
-        this.op = op;
-    }
-
-    @Override
-    public ExpressionTag getTag() {
-        return ExpressionTag.RELATIONAL;
-    }
-
-    public ConstraintExpression getLeft() {
-        return left;
-    }
-
-    public ConstraintExpression getRight() {
-        return right;
-    }
-
-    public Operator getOperator() {
-        return op;
-    }
-
-    @Override
-    public void getChildren(Collection<ConstraintExpression> children) {
-        children.add(left);
-        children.add(right);
-    }
-
-    @Override
-    protected void toString(StringBuilder buffer) {
-        buffer.append(getTag()).append('(').append(op).append(", ");
-        left.toString(buffer);
-        buffer.append(", ");
-        right.toString(buffer);
-        buffer.append(')');
-    }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ResourceSumExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ResourceSumExpression.java
deleted file mode 100644
index 3fbdd08..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ResourceSumExpression.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.constraints.expressions;
-
-import java.util.Collection;
-
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-
-public final class ResourceSumExpression extends ConstraintExpression {
-    private static final long serialVersionUID = 1L;
-
-    private final OperatorDescriptorId opId;
-
-    private final String resourceName;
-
-    public ResourceSumExpression(OperatorDescriptorId opId, String resourceName) {
-        this.opId = opId;
-        this.resourceName = resourceName;
-    }
-
-    @Override
-    public ExpressionTag getTag() {
-        return ExpressionTag.RESOURCE_SUM;
-    }
-
-    public OperatorDescriptorId getOperatorDescriptorId() {
-        return opId;
-    }
-
-    public String getResourceName() {
-        return resourceName;
-    }
-
-    @Override
-    public void getChildren(Collection<ConstraintExpression> children) {
-    }
-
-    @Override
-    protected void toString(StringBuilder buffer) {
-        buffer.append(getTag()).append('(').append(opId.toString()).append(", ").append(resourceName).append(')');
-    }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java
index 534507e..e3c71ea 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java
@@ -14,19 +14,16 @@
  */
 package edu.uci.ics.hyracks.api.context;
 
-import java.util.UUID;
-
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
 import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
 
 public interface IHyracksJobletContext extends IHyracksCommonContext, IWorkspaceFileFactory, IDeallocatableRegistry {
     public INCApplicationContext getApplicationContext();
 
-    public UUID getJobId();
-
-    public int getAttempt();
+    public JobId getJobId();
 
     public ICounterContext getCounterContext();
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksStageletContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksStageletContext.java
deleted file mode 100644
index ae171fe..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksStageletContext.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.context;
-
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
-import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
-
-public interface IHyracksStageletContext extends IHyracksCommonContext, IWorkspaceFileFactory, IDeallocatableRegistry {
-    public IHyracksJobletContext getJobletContext();
-
-    public UUID getStageId();
-
-    public ICounterContext getCounterContext();
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
new file mode 100644
index 0000000..7609951
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.context;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
+import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
+
+public interface IHyracksTaskContext extends IHyracksCommonContext, IWorkspaceFileFactory, IDeallocatableRegistry {
+    public IHyracksJobletContext getJobletContext();
+
+    public TaskAttemptId getTaskAttemptId();
+
+    public ICounterContext getCounterContext();
+
+    public Object lookupGlobalVariable(ActivityId producerActivity, int partition, String varName)
+            throws HyracksDataException;
+
+    public Object lookupLocalVariable(ActivityId producerActivity, int partition, String varName)
+            throws HyracksDataException;
+
+    public void setVariable(String name, Object value) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/IClusterController.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/IClusterController.java
deleted file mode 100644
index 9023cee..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/IClusterController.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.control;
-
-import java.rmi.Remote;
-import java.util.List;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.api.job.profiling.om.StageletProfile;
-
-public interface IClusterController extends Remote {
-    public NodeParameters registerNode(INodeController nodeController) throws Exception;
-
-    public void unregisterNode(INodeController nodeController) throws Exception;
-
-    public void notifyStageletComplete(UUID jobId, UUID stageId, int attempt, String nodeId, StageletProfile statistics)
-            throws Exception;
-
-    public void notifyStageletFailure(UUID jobId, UUID stageId, int attempt, String nodeId) throws Exception;
-
-    public void nodeHeartbeat(String id) throws Exception;
-
-    public void reportProfile(String id, List<JobProfile> profiles) throws Exception;
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
deleted file mode 100644
index d0269fc..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.control;
-
-import java.rmi.Remote;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
-
-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(String appName, UUID jobId, int attempt, byte[] plan,
-            UUID stageId, Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Integer> opNumPartitions)
-            throws Exception;
-
-    public void initializeJobletPhase2(String appName, UUID jobId, byte[] plan, UUID stageId,
-            Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Integer> opNumPartitions,
-            Map<PortInstanceId, Endpoint> globalPortMap) throws Exception;
-
-    public void commitJobletInitialization(UUID jobId, UUID stageId) throws Exception;
-
-    public void abortJoblet(UUID jobId, int attempt) 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;
-
-    public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
-            throws Exception;
-
-    public void destroyApplication(String appName) throws Exception;
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityNodeId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
similarity index 73%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityNodeId.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
index 12f76fe..c6ca51c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityNodeId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
@@ -15,14 +15,13 @@
 package edu.uci.ics.hyracks.api.dataflow;
 
 import java.io.Serializable;
-import java.util.UUID;
 
-public final class ActivityNodeId implements Serializable {
+public final class ActivityId implements Serializable {
     private static final long serialVersionUID = 1L;
     private final OperatorDescriptorId odId;
-    private final UUID id;
+    private final long id;
 
-    public ActivityNodeId(OperatorDescriptorId odId, UUID id) {
+    public ActivityId(OperatorDescriptorId odId, long id) {
         this.odId = odId;
         this.id = id;
     }
@@ -31,13 +30,13 @@
         return odId;
     }
 
-    public UUID getLocalId() {
+    public long getLocalId() {
         return id;
     }
 
     @Override
     public int hashCode() {
-        return odId.hashCode() + id.hashCode();
+        return (int) (odId.hashCode() + id);
     }
 
     @Override
@@ -45,14 +44,14 @@
         if (this == o) {
             return true;
         }
-        if (!(o instanceof ActivityNodeId)) {
+        if (!(o instanceof ActivityId)) {
             return false;
         }
-        ActivityNodeId other = (ActivityNodeId) o;
-        return other.odId.equals(odId) && other.id.equals(id);
+        ActivityId other = (ActivityId) o;
+        return other.odId.equals(odId) && other.id == id;
     }
 
     public String toString() {
-        return "ANID:" + id;
+        return "ANID:[" + odId + "]:" + id;
     }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java
index 86d8118..3133a7f 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java
@@ -15,24 +15,23 @@
 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;
+    private int id;
 
-    public ConnectorDescriptorId(UUID id) {
+    public ConnectorDescriptorId(int id) {
         this.id = id;
     }
 
-    public UUID getId() {
+    public int getId() {
         return id;
     }
 
     @Override
     public int hashCode() {
-        return id.hashCode();
+        return id;
     }
 
     @Override
@@ -44,7 +43,7 @@
             return false;
         }
         ConnectorDescriptorId other = (ConnectorDescriptorId) obj;
-        return id.equals(other.id);
+        return id == other.id;
     }
 
     @Override
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivity.java
similarity index 76%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivity.java
index 39d2eda..6b519d6 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivity.java
@@ -16,16 +16,14 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 
-public interface IActivityNode extends Serializable {
-    public ActivityNodeId getActivityNodeId();
+public interface IActivity extends Serializable {
+    public ActivityId getActivityId();
 
-    public IOperatorDescriptor getOwner();
-
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java
index c6bc5e3..56870b2 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java
@@ -15,11 +15,11 @@
 package edu.uci.ics.hyracks.api.dataflow;
 
 public interface IActivityGraphBuilder {
-    public void addTask(IActivityNode task);
+    public void addActivity(IActivity task);
 
-    public void addBlockingEdge(IActivityNode blocker, IActivityNode blocked);
+    public void addBlockingEdge(IActivity blocker, IActivity blocked);
 
-    public void addSourceEdge(int operatorInputIndex, IActivityNode task, int taskInputIndex);
+    public void addSourceEdge(int operatorInputIndex, IActivity task, int taskInputIndex);
 
-    public void addTargetEdge(int operatorOutputIndex, IActivityNode task, int taskOutputIndex);
+    public void addTargetEdge(int operatorOutputIndex, IActivity task, int taskOutputIndex);
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
index 3dcf620..ec8ab76 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
@@ -15,19 +15,20 @@
 package edu.uci.ics.hyracks.api.dataflow;
 
 import java.io.Serializable;
+import java.util.BitSet;
 
 import org.json.JSONException;
 import org.json.JSONObject;
 
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-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.constraints.IConstraintExpressionAcceptor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
 
 /**
  * Connector that connects operators in a Job.
@@ -60,8 +61,8 @@
      * @return data writer.
      * @throws Exception
      */
-    public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
-            IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException;
 
     /**
@@ -71,20 +72,17 @@
      *            Context
      * @param recordDesc
      *            Job plan
-     * @param demux
-     *            Connection Demultiplexer
-     * @param index
+     * @param receiverIndex
      *            ordinal index of the data consumer partition
      * @param nProducerPartitions
      *            Number of partitions of the producing operator.
      * @param nConsumerPartitions
      *            Number of partitions of the consuming operator.
-     * @return data reader
+     * @return partition collector
      * @throws HyracksDataException
      */
-    public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
-            IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
-            throws HyracksDataException;
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            int receiverIndex, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException;
 
     /**
      * Contribute any scheduling constraints imposed by this connector
@@ -94,10 +92,16 @@
      * @param plan
      *            - Job Plan
      */
-    public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan,
+    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
             ICCApplicationContext appCtx);
 
     /**
+     * Indicate which consumer partitions may receive data from the given producer partition.
+     */
+    public void indicateTargetPartitions(int nProducerPartitions, int nConsumerPartitions, int producerIndex,
+            BitSet targetBitmap);
+
+    /**
      * Translate this connector descriptor to JSON.
      * 
      * @return
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
index 9e8dbcb..78847eb 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
@@ -20,9 +20,9 @@
 import org.json.JSONObject;
 
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
 
 /**
  * Descriptor for operators in Hyracks.
@@ -64,7 +64,7 @@
      * @param builder
      *            - graph builder
      */
-    public void contributeTaskGraph(IActivityGraphBuilder builder);
+    public void contributeActivities(IActivityGraphBuilder builder);
 
     /**
      * Contributes any scheduling constraints imposed by this operator.
@@ -74,7 +74,7 @@
      * @param plan
      *            - Job Plan
      */
-    public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan,
+    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
             ICCApplicationContext appCtx);
 
     /**
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
index bbf3cf5..b858736 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
@@ -15,24 +15,23 @@
 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;
+    private final int id;
 
-    public OperatorDescriptorId(UUID id) {
+    public OperatorDescriptorId(int id) {
         this.id = id;
     }
 
-    public UUID getId() {
+    public int getId() {
         return id;
     }
 
     @Override
     public int hashCode() {
-        return id.hashCode();
+        return id;
     }
 
     @Override
@@ -43,7 +42,7 @@
         if (!(o instanceof OperatorDescriptorId)) {
             return false;
         }
-        return ((OperatorDescriptorId) o).id.equals(id);
+        return ((OperatorDescriptorId) o).id == id;
     }
 
     @Override
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/PortInstanceId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/PortInstanceId.java
deleted file mode 100644
index 844266d..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/PortInstanceId.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.dataflow;
-
-import java.io.Serializable;
-
-public final class PortInstanceId implements Serializable {
-    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-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java
new file mode 100644
index 0000000..d4e6972
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.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.api.dataflow;
+
+import java.io.Serializable;
+
+public final class TaskAttemptId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final TaskId taskId;
+
+    private final int attempt;
+
+    public TaskAttemptId(TaskId taskId, int attempt) {
+        this.taskId = taskId;
+        this.attempt = attempt;
+    }
+
+    public TaskId getTaskId() {
+        return taskId;
+    }
+
+    public int getAttempt() {
+        return attempt;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof TaskAttemptId)) {
+            return false;
+        }
+        TaskAttemptId oTaskId = (TaskAttemptId) o;
+        return oTaskId.attempt == attempt && oTaskId.taskId.equals(taskId);
+    }
+
+    @Override
+    public int hashCode() {
+        return taskId.hashCode() + attempt;
+    }
+
+    @Override
+    public String toString() {
+        return "TAID:[" + taskId + "]:" + attempt;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java
new file mode 100644
index 0000000..ee63355
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.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.api.dataflow;
+
+import java.io.Serializable;
+
+public final class TaskId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final ActivityId activityId;
+
+    private final int partition;
+
+    public TaskId(ActivityId activityId, int partition) {
+        this.activityId = activityId;
+        this.partition = partition;
+    }
+
+    public ActivityId getActivityId() {
+        return activityId;
+    }
+
+    public int getPartition() {
+        return partition;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof TaskId)) {
+            return false;
+        }
+        TaskId oTaskId = (TaskId) o;
+        return oTaskId.partition == partition && oTaskId.activityId.equals(activityId);
+    }
+
+    @Override
+    public int hashCode() {
+        return activityId.hashCode() + partition;
+    }
+
+    @Override
+    public String toString() {
+        return "TID:[" + activityId + "]:" + partition;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/IConnectorPolicy.java
similarity index 67%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/IConnectorPolicy.java
index 88efc1f..d6d6fee 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/IConnectorPolicy.java
@@ -12,20 +12,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.api.dataflow.connectors;
 
 import java.io.Serializable;
 
-public class NodeCapability implements Serializable {
-    private static final long serialVersionUID = 1L;
+public interface IConnectorPolicy extends Serializable {
+    public boolean requiresProducerConsumerCoscheduling();
 
-    private int cpuCount;
+    public boolean consumerWaitsForProducerToFinish();
 
-    public int getCPUCount() {
-        return cpuCount;
-    }
+    public boolean materializeOnSendSide();
 
-    public void setCPUCount(int cpuCount) {
-        this.cpuCount = cpuCount;
-    }
+    public boolean materializeOnReceiveSide();
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/IConnectorPolicyAssignmentPolicy.java
similarity index 66%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/IConnectorPolicyAssignmentPolicy.java
index 88efc1f..ca684e4 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/IConnectorPolicyAssignmentPolicy.java
@@ -12,20 +12,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.api.dataflow.connectors;
 
 import java.io.Serializable;
 
-public class NodeCapability implements Serializable {
-    private static final long serialVersionUID = 1L;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 
-    private int cpuCount;
-
-    public int getCPUCount() {
-        return cpuCount;
-    }
-
-    public void setCPUCount(int cpuCount) {
-        this.cpuCount = cpuCount;
-    }
+public interface IConnectorPolicyAssignmentPolicy extends Serializable {
+    public IConnectorPolicy getConnectorPolicyAssignment(IConnectorDescriptor c, int nProducers, int nConsumers,
+            int[] fanouts);
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/PipeliningConnectorPolicy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/PipeliningConnectorPolicy.java
new file mode 100644
index 0000000..39b3904
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/PipeliningConnectorPolicy.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.connectors;
+
+public final class PipeliningConnectorPolicy implements IConnectorPolicy {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public boolean requiresProducerConsumerCoscheduling() {
+        return true;
+    }
+
+    @Override
+    public boolean consumerWaitsForProducerToFinish() {
+        return true;
+    }
+
+    @Override
+    public boolean materializeOnSendSide() {
+        return false;
+    }
+
+    @Override
+    public boolean materializeOnReceiveSide() {
+        return false;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedBlockingConnectorPolicy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedBlockingConnectorPolicy.java
new file mode 100644
index 0000000..a5017de
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedBlockingConnectorPolicy.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.connectors;
+
+public final class SendSideMaterializedBlockingConnectorPolicy implements IConnectorPolicy {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public boolean requiresProducerConsumerCoscheduling() {
+        return false;
+    }
+
+    @Override
+    public boolean consumerWaitsForProducerToFinish() {
+        return true;
+    }
+
+    @Override
+    public boolean materializeOnSendSide() {
+        return true;
+    }
+
+    @Override
+    public boolean materializeOnReceiveSide() {
+        return false;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedPipeliningConnectorPolicy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedPipeliningConnectorPolicy.java
new file mode 100644
index 0000000..5d522c0
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedPipeliningConnectorPolicy.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.connectors;
+
+public final class SendSideMaterializedPipeliningConnectorPolicy implements IConnectorPolicy {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public boolean requiresProducerConsumerCoscheduling() {
+        return false;
+    }
+
+    @Override
+    public boolean consumerWaitsForProducerToFinish() {
+        return false;
+    }
+
+    @Override
+    public boolean materializeOnSendSide() {
+        return true;
+    }
+
+    @Override
+    public boolean materializeOnReceiveSide() {
+        return false;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy.java
new file mode 100644
index 0000000..82608c4
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy.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.connectors;
+
+public final class SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy implements IConnectorPolicy {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public boolean requiresProducerConsumerCoscheduling() {
+        return false;
+    }
+
+    @Override
+    public boolean consumerWaitsForProducerToFinish() {
+        return true;
+    }
+
+    @Override
+    public boolean materializeOnSendSide() {
+        return true;
+    }
+
+    @Override
+    public boolean materializeOnReceiveSide() {
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedReceiveSideMaterializedPipeliningConnectorPolicy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedReceiveSideMaterializedPipeliningConnectorPolicy.java
new file mode 100644
index 0000000..1cc0583
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedReceiveSideMaterializedPipeliningConnectorPolicy.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.connectors;
+
+public final class SendSideMaterializedReceiveSideMaterializedPipeliningConnectorPolicy implements IConnectorPolicy {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public boolean requiresProducerConsumerCoscheduling() {
+        return false;
+    }
+
+    @Override
+    public boolean consumerWaitsForProducerToFinish() {
+        return false;
+    }
+
+    @Override
+    public boolean materializeOnSendSide() {
+        return true;
+    }
+
+    @Override
+    public boolean materializeOnReceiveSide() {
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java
index e9ba2c3..9e27077 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java
@@ -51,4 +51,8 @@
     public int hashCode() {
         return file.hashCode();
     }
+
+    public void delete() {
+        file.delete();
+    }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWorkspaceFileFactory.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWorkspaceFileFactory.java
index 1311ec5..7b9a7c4 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWorkspaceFileFactory.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWorkspaceFileFactory.java
@@ -17,5 +17,7 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface IWorkspaceFileFactory {
-    public FileReference createWorkspaceFile(String prefix) throws HyracksDataException;
+    public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException;
+
+    public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobLifecycleListener.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobLifecycleListener.java
index 008fdb1..c3a0f7e 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobLifecycleListener.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobLifecycleListener.java
@@ -14,14 +14,12 @@
  */
 package edu.uci.ics.hyracks.api.job;
 
-import java.util.UUID;
-
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 
 public interface IJobLifecycleListener {
-    public void notifyJobCreation(UUID jobId, JobSpecification jobSpec) throws HyracksException;
+    public void notifyJobCreation(JobId jobId, JobSpecification jobSpec) throws HyracksException;
 
-    public void notifyJobStart(UUID jobId) throws HyracksException;
+    public void notifyJobStart(JobId jobId) throws HyracksException;
 
-    public void notifyJobFinish(UUID jobId) throws HyracksException;
+    public void notifyJobFinish(JobId jobId) throws HyracksException;
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
new file mode 100644
index 0000000..930d299
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
@@ -0,0 +1,263 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.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 JobActivityGraph implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final String appName;
+
+    private final JobSpecification jobSpec;
+
+    private final EnumSet<JobFlag> jobFlags;
+
+    private final Map<ActivityId, IActivity> activityNodes;
+
+    private final Map<ActivityId, Set<ActivityId>> blocker2blockedMap;
+
+    private final Map<ActivityId, Set<ActivityId>> blocked2blockerMap;
+
+    private final Map<OperatorDescriptorId, Set<ActivityId>> operatorActivityMap;
+
+    private final Map<ActivityId, List<Integer>> activityInputMap;
+
+    private final Map<ActivityId, List<Integer>> activityOutputMap;
+
+    private final Map<OperatorDescriptorId, List<ActivityId>> operatorInputMap;
+
+    private final Map<OperatorDescriptorId, List<ActivityId>> operatorOutputMap;
+
+    public JobActivityGraph(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
+        this.appName = appName;
+        this.jobSpec = jobSpec;
+        this.jobFlags = jobFlags;
+        activityNodes = new HashMap<ActivityId, IActivity>();
+        blocker2blockedMap = new HashMap<ActivityId, Set<ActivityId>>();
+        blocked2blockerMap = new HashMap<ActivityId, Set<ActivityId>>();
+        operatorActivityMap = new HashMap<OperatorDescriptorId, Set<ActivityId>>();
+        activityInputMap = new HashMap<ActivityId, List<Integer>>();
+        activityOutputMap = new HashMap<ActivityId, List<Integer>>();
+        operatorInputMap = new HashMap<OperatorDescriptorId, List<ActivityId>>();
+        operatorOutputMap = new HashMap<OperatorDescriptorId, List<ActivityId>>();
+    }
+
+    public String getApplicationName() {
+        return appName;
+    }
+
+    public JobSpecification getJobSpecification() {
+        return jobSpec;
+    }
+
+    public EnumSet<JobFlag> getJobFlags() {
+        return jobFlags;
+    }
+
+    public Map<ActivityId, IActivity> getActivityNodeMap() {
+        return activityNodes;
+    }
+
+    public Map<ActivityId, Set<ActivityId>> getBlocker2BlockedMap() {
+        return blocker2blockedMap;
+    }
+
+    public Map<ActivityId, Set<ActivityId>> getBlocked2BlockerMap() {
+        return blocked2blockerMap;
+    }
+
+    public Map<OperatorDescriptorId, Set<ActivityId>> getOperatorActivityMap() {
+        return operatorActivityMap;
+    }
+
+    public Map<ActivityId, List<Integer>> getActivityInputMap() {
+        return activityInputMap;
+    }
+
+    public Map<ActivityId, List<Integer>> getActivityOutputMap() {
+        return activityOutputMap;
+    }
+
+    public Map<OperatorDescriptorId, List<ActivityId>> getOperatorInputMap() {
+        return operatorInputMap;
+    }
+
+    public Map<OperatorDescriptorId, List<ActivityId>> getOperatorOutputMap() {
+        return operatorOutputMap;
+    }
+
+    public List<IConnectorDescriptor> getActivityInputConnectorDescriptors(ActivityId hanId) {
+        List<Integer> inputIndexes = activityInputMap.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> getActivityOutputConnectorDescriptors(ActivityId hanId) {
+        List<Integer> outputIndexes = activityOutputMap.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 ActivityId getConsumerActivity(ConnectorDescriptorId cdId) {
+        Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connEdge = jobSpec
+                .getConnectorOperatorMap().get(cdId);
+
+        OperatorDescriptorId consumerOpId = connEdge.second.first.getOperatorId();
+        int consumerInputIdx = connEdge.second.second;
+
+        for (ActivityId anId : operatorActivityMap.get(consumerOpId)) {
+            List<Integer> anInputs = activityInputMap.get(anId);
+            if (anInputs != null) {
+                for (Integer idx : anInputs) {
+                    if (idx.intValue() == consumerInputIdx) {
+                        return anId;
+                    }
+                }
+            }
+        }
+        return null;
+    }
+
+    public ActivityId getProducerActivity(ConnectorDescriptorId cdId) {
+        Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connEdge = jobSpec
+                .getConnectorOperatorMap().get(cdId);
+
+        OperatorDescriptorId producerOpId = connEdge.first.first.getOperatorId();
+        int producerInputIdx = connEdge.first.second;
+
+        for (ActivityId anId : operatorActivityMap.get(producerOpId)) {
+            List<Integer> anOutputs = activityOutputMap.get(anId);
+            if (anOutputs != null) {
+                for (Integer idx : anOutputs) {
+                    if (idx.intValue() == producerInputIdx) {
+                        return anId;
+                    }
+                }
+            }
+        }
+        return null;
+    }
+
+    public RecordDescriptor getActivityInputRecordDescriptor(ActivityId hanId, int inputIndex) {
+        int opInputIndex = getActivityInputMap().get(hanId).get(inputIndex);
+        return jobSpec.getOperatorInputRecordDescriptor(hanId.getOperatorDescriptorId(), opInputIndex);
+    }
+
+    public RecordDescriptor getActivityOutputRecordDescriptor(ActivityId hanId, int outputIndex) {
+        int opOutputIndex = getActivityOutputMap().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();
+    }
+
+    public JSONObject toJSON() throws JSONException {
+        JSONObject jplan = new JSONObject();
+
+        jplan.put("type", "plan");
+        jplan.put("flags", jobFlags.toString());
+
+        JSONArray jans = new JSONArray();
+        for (IActivity an : activityNodes.values()) {
+            JSONObject jan = new JSONObject();
+            jan.put("type", "activity");
+            jan.put("id", an.getActivityId().toString());
+            jan.put("java-class", an.getClass().getName());
+            jan.put("operator-id", an.getActivityId().getOperatorDescriptorId().toString());
+
+            List<IConnectorDescriptor> inputs = getActivityInputConnectorDescriptors(an.getActivityId());
+            if (inputs != null) {
+                JSONArray jInputs = new JSONArray();
+                for (int i = 0; i < inputs.size(); ++i) {
+                    JSONObject jInput = new JSONObject();
+                    jInput.put("type", "activity-input");
+                    jInput.put("input-port", i);
+                    jInput.put("connector-id", inputs.get(i).getConnectorId().toString());
+                    jInputs.put(jInput);
+                }
+                jan.put("inputs", jInputs);
+            }
+
+            List<IConnectorDescriptor> outputs = getActivityOutputConnectorDescriptors(an.getActivityId());
+            if (outputs != null) {
+                JSONArray jOutputs = new JSONArray();
+                for (int i = 0; i < outputs.size(); ++i) {
+                    JSONObject jOutput = new JSONObject();
+                    jOutput.put("type", "activity-output");
+                    jOutput.put("output-port", i);
+                    jOutput.put("connector-id", outputs.get(i).getConnectorId().toString());
+                    jOutputs.put(jOutput);
+                }
+                jan.put("outputs", jOutputs);
+            }
+
+            Set<ActivityId> blockers = getBlocked2BlockerMap().get(an.getActivityId());
+            if (blockers != null) {
+                JSONArray jDeps = new JSONArray();
+                for (ActivityId blocker : blockers) {
+                    jDeps.put(blocker.toString());
+                }
+                jan.put("depends-on", jDeps);
+            }
+            jans.put(jan);
+        }
+        jplan.put("activities", jans);
+
+        return jplan;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobId.java
new file mode 100644
index 0000000..ffb9bba
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobId.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.api.job;
+
+import java.io.Serializable;
+
+public final class JobId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final long id;
+
+    public JobId(long id) {
+        this.id = id;
+    }
+
+    public long getId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int) id;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof JobId)) {
+            return false;
+        }
+        return ((JobId) o).id == id;
+    }
+
+    @Override
+    public String toString() {
+        return "JID:" + id;
+    }
+
+    public static JobId parse(String str) {
+        if (str.startsWith("JID:")) {
+            str = str.substring(4);
+            return new JobId(Long.parseLong(str));
+        }
+        throw new IllegalArgumentException();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobPlan.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobPlan.java
deleted file mode 100644
index 13da402..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobPlan.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.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 java.util.UUID;
-
-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 String appName;
-
-    private final UUID jobId;
-
-    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;
-
-    public JobPlan(String appName, UUID jobId, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
-        this.appName = appName;
-        this.jobId = jobId;
-        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 String getApplicationName() {
-        return appName;
-    }
-
-    public UUID getJobId() {
-        return jobId;
-    }
-
-    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 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-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
index 59072f8..e3d28ea 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
@@ -26,11 +26,12 @@
 import org.json.JSONException;
 import org.json.JSONObject;
 
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
 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.connectors.IConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.util.Pair;
 
@@ -51,10 +52,16 @@
 
     private final Map<String, Serializable> properties;
 
-    private final Set<ConstraintExpression> userConstraints;
+    private final Set<Constraint> userConstraints;
+
+    private IConnectorPolicyAssignmentPolicy connectorPolicyAssignmentPolicy;
 
     private int maxAttempts;
 
+    private transient int operatorIdCounter;
+
+    private transient int connectorIdCounter;
+
     public JobSpecification() {
         roots = new ArrayList<OperatorDescriptorId>();
         opMap = new HashMap<OperatorDescriptorId, IOperatorDescriptor>();
@@ -63,7 +70,17 @@
         opOutputMap = new HashMap<OperatorDescriptorId, List<IConnectorDescriptor>>();
         connectorOpMap = new HashMap<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>>();
         properties = new HashMap<String, Serializable>();
-        userConstraints = new HashSet<ConstraintExpression>();
+        userConstraints = new HashSet<Constraint>();
+        operatorIdCounter = 0;
+        connectorIdCounter = 0;
+    }
+
+    public OperatorDescriptorId createOperatorDescriptorId() {
+        return new OperatorDescriptorId(operatorIdCounter++);
+    }
+
+    public ConnectorDescriptorId createConnectorDescriptor() {
+        return new ConnectorDescriptorId(connectorIdCounter++);
     }
 
     public void addRoot(IOperatorDescriptor op) {
@@ -173,6 +190,14 @@
         return roots;
     }
 
+    public IConnectorPolicyAssignmentPolicy getConnectorPolicyAssignmentPolicy() {
+        return connectorPolicyAssignmentPolicy;
+    }
+
+    public void setConnectorPolicyAssignmentPolicy(IConnectorPolicyAssignmentPolicy connectorPolicyAssignmentPolicy) {
+        this.connectorPolicyAssignmentPolicy = connectorPolicyAssignmentPolicy;
+    }
+
     public void setMaxAttempts(int maxAttempts) {
         this.maxAttempts = maxAttempts;
     }
@@ -181,11 +206,11 @@
         return maxAttempts;
     }
 
-    public void addUserConstraint(ConstraintExpression constraint) {
+    public void addUserConstraint(Constraint constraint) {
         userConstraints.add(constraint);
     }
 
-    public Set<ConstraintExpression> getUserConstraints() {
+    public Set<Constraint> getUserConstraints() {
         return userConstraints;
     }
 
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/StageletProfile.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/StageletProfile.java
deleted file mode 100644
index cb43f66..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/StageletProfile.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.job.profiling.om;
-
-import java.util.UUID;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-
-public class StageletProfile extends AbstractProfile {
-    private static final long serialVersionUID = 1L;
-
-    private final UUID stageId;
-
-    public StageletProfile(UUID stageId) {
-        this.stageId = stageId;
-    }
-
-    public UUID getStageId() {
-        return stageId;
-    }
-
-    @Override
-    public JSONObject toJSON() throws JSONException {
-        JSONObject json = new JSONObject();
-
-        json.put("type", "joblet-profile");
-        json.put("stage-id", stageId.toString());
-        populateCounters(json);
-
-        return json;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/naming/MultipartName.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/naming/MultipartName.java
new file mode 100644
index 0000000..7310606
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/naming/MultipartName.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.naming;
+
+import java.util.Arrays;
+
+public final class MultipartName {
+    private Object[] parts;
+
+    public MultipartName(Object... parts) {
+        this.parts = parts;
+    }
+
+    public Object[] getParts() {
+        return parts;
+    }
+
+    @Override
+    public int hashCode() {
+        return Arrays.deepHashCode(parts);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof MultipartName)) {
+            return false;
+        }
+        return Arrays.deepEquals(parts, ((MultipartName) o).parts);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/IPartition.java
similarity index 74%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/IPartition.java
index 45c589a..02eb891 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/IPartition.java
@@ -12,11 +12,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.partitions;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.resources.IDeallocatable;
 
-public interface IEndpointDataWriterFactory {
-    public IFrameWriter createFrameWriter(int index) throws HyracksDataException;
+public interface IPartition extends IDeallocatable {
+    public void writeTo(IFrameWriter writer);
+
+    public boolean isReusable();
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/PartitionId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/PartitionId.java
new file mode 100644
index 0000000..be25d12
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/PartitionId.java
@@ -0,0 +1,97 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.partitions;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public final class PartitionId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final JobId jobId;
+
+    private final ConnectorDescriptorId cdId;
+
+    private final int senderIndex;
+
+    private final int receiverIndex;
+
+    public PartitionId(JobId jobId, ConnectorDescriptorId cdId, int senderIndex, int receiverIndex) {
+        this.jobId = jobId;
+        this.cdId = cdId;
+        this.senderIndex = senderIndex;
+        this.receiverIndex = receiverIndex;
+    }
+
+    public JobId getJobId() {
+        return jobId;
+    }
+
+    public ConnectorDescriptorId getConnectorDescriptorId() {
+        return cdId;
+    }
+
+    public int getSenderIndex() {
+        return senderIndex;
+    }
+
+    public int getReceiverIndex() {
+        return receiverIndex;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((cdId == null) ? 0 : cdId.hashCode());
+        result = prime * result + ((jobId == null) ? 0 : jobId.hashCode());
+        result = prime * result + receiverIndex;
+        result = prime * result + senderIndex;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        PartitionId other = (PartitionId) obj;
+        if (cdId == null) {
+            if (other.cdId != null)
+                return false;
+        } else if (!cdId.equals(other.cdId))
+            return false;
+        if (jobId == null) {
+            if (other.jobId != null)
+                return false;
+        } else if (!jobId.equals(other.jobId))
+            return false;
+        if (receiverIndex != other.receiverIndex)
+            return false;
+        if (senderIndex != other.senderIndex)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return jobId.toString() + ":" + cdId + ":" + senderIndex + ":" + receiverIndex;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/ISpaceSharedResource.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/ISpaceSharedResource.java
deleted file mode 100644
index 3a6f930..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/ISpaceSharedResource.java
+++ /dev/null
@@ -1,19 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.resources;
-
-public interface ISpaceSharedResource {
-
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/ITimeSharedResource.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/ITimeSharedResource.java
deleted file mode 100644
index 1fb0465..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/ITimeSharedResource.java
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.resources;
-
-public interface ITimeSharedResource {
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/ISliver.java
similarity index 69%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/ISliver.java
index 45c589a..135a77c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/ISliver.java
@@ -12,11 +12,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.things;
 
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import java.nio.ByteBuffer;
+
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
-public interface IEndpointDataWriterFactory {
-    public IFrameWriter createFrameWriter(int index) throws HyracksDataException;
+public interface ISliver {
+    public void open() throws HyracksDataException;
+
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException;
+
+    public void commit() throws HyracksDataException;
+
+    public void abort();
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResource.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/IThingDescriptor.java
similarity index 82%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResource.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/IThingDescriptor.java
index 2d76c26..1269e35 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResource.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/IThingDescriptor.java
@@ -12,9 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.resources;
+package edu.uci.ics.hyracks.api.things;
 
 import java.io.Serializable;
 
-public interface IResource extends Serializable {
+public interface IThingDescriptor extends Serializable {
+    public ThingDescriptorId getThingId();
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/IThingPartition.java
similarity index 84%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/IThingPartition.java
index c884998..cb79b6f 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/IThingPartition.java
@@ -12,9 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.things;
 
-public enum Direction {
-    INPUT,
-    OUTPUT,
+public interface IThingPartition {
+    public ISliver createSliver();
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/ThingDescriptorId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/ThingDescriptorId.java
new file mode 100644
index 0000000..5d42ce9
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/ThingDescriptorId.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.things;
+
+import java.io.Serializable;
+
+public final class ThingDescriptorId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final long id;
+
+    public ThingDescriptorId(long id) {
+        this.id = id;
+    }
+
+    public long getId() {
+        return id;
+    }
+
+    @Override
+    public String toString() {
+        return "ThID: " + id;
+    }
+
+    public int hashCode() {
+        return (int) (id & 0xffffffff);
+    }
+
+    public boolean equals(Object o) {
+        if (!(o instanceof ThingDescriptorId)) {
+            return false;
+        }
+        return id == ((ThingDescriptorId) o).id;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java
index 1451c69..1185b5e 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java
@@ -34,6 +34,9 @@
     }
 
     public static Object deserialize(byte[] bytes, ClassLoader classLoader) throws IOException, ClassNotFoundException {
+        if (bytes == null) {
+            return null;
+        }
         ClassLoader ctxCL = Thread.currentThread().getContextClassLoader();
         try {
             Thread.currentThread().setContextClassLoader(classLoader);
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableDescriptor.java
new file mode 100644
index 0000000..b13afe4
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableDescriptor.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.workflow.variables;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.naming.MultipartName;
+
+public final class WorkflowVariableDescriptor implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final MultipartName name;
+
+    private final WorkflowVariableValueScope scope;
+
+    public WorkflowVariableDescriptor(MultipartName name, WorkflowVariableValueScope scope) {
+        this.name = name;
+        this.scope = scope;
+    }
+
+    public MultipartName getName() {
+        return name;
+    }
+
+    public WorkflowVariableValueScope getScope() {
+        return scope;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableValueScope.java
similarity index 84%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableValueScope.java
index c884998..44ed129 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableValueScope.java
@@ -12,9 +12,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.workflow.variables;
 
-public enum Direction {
-    INPUT,
-    OUTPUT,
+public enum WorkflowVariableValueScope {
+    LOCAL,
+    GLOBAL,
 }
\ No newline at end of file
diff --git a/hyracks-cli/.classpath b/hyracks-cli/.classpath
index 93843eb..ba0bb5a 100644
--- a/hyracks-cli/.classpath
+++ b/hyracks-cli/.classpath
@@ -1,7 +1,7 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <classpath>
 	<classpathentry kind="src" output="target/classes" path="src/main/java"/>
-	<classpathentry kind="src" output="target/classes" path="target/generated-sources/javacc"/>
+	<classpathentry kind="src" path="target/generated-sources/javacc"/>
 	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/JavaSE-1.6"/>
 	<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
 	<classpathentry kind="output" path="target/classes"/>
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCClientInterface.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCClientInterface.java
new file mode 100644
index 0000000..77694b4
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCClientInterface.java
@@ -0,0 +1,61 @@
+package edu.uci.ics.hyracks.control.cc;
+
+import java.rmi.RemoteException;
+import java.rmi.server.UnicastRemoteObject;
+import java.util.EnumSet;
+
+import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
+import edu.uci.ics.hyracks.api.client.IHyracksClientInterface;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+
+public class CCClientInterface extends UnicastRemoteObject implements IHyracksClientInterface {
+    private static final long serialVersionUID = 1L;
+
+    private final ClusterControllerService ccs;
+
+    public CCClientInterface(ClusterControllerService ccs) throws RemoteException {
+        this.ccs = ccs;
+    }
+
+    @Override
+    public ClusterControllerInfo getClusterControllerInfo() throws Exception {
+        return ccs.getClusterControllerInfo();
+    }
+
+    @Override
+    public void createApplication(String appName) throws Exception {
+        ccs.createApplication(appName);
+    }
+
+    @Override
+    public void startApplication(String appName) throws Exception {
+        ccs.startApplication(appName);
+    }
+
+    @Override
+    public void destroyApplication(String appName) throws Exception {
+        ccs.destroyApplication(appName);
+    }
+
+    @Override
+    public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+        return ccs.createJob(appName, jobSpec, jobFlags);
+    }
+
+    @Override
+    public JobStatus getJobStatus(JobId jobId) throws Exception {
+        return ccs.getJobStatus(jobId);
+    }
+
+    @Override
+    public void start(JobId jobId) throws Exception {
+        ccs.start(jobId);
+    }
+
+    @Override
+    public void waitForCompletion(JobId jobId) throws Exception {
+        ccs.waitForCompletion(jobId);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCDriver.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCDriver.java
index 3e36a37..4bf3da2 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCDriver.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCDriver.java
@@ -16,7 +16,7 @@
 
 import org.kohsuke.args4j.CmdLineParser;
 
-import edu.uci.ics.hyracks.api.control.CCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
 
 public class CCDriver {
     public static void main(String args[]) throws Exception {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index e5efe92..d864c05 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -26,7 +26,6 @@
 import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
-import java.util.UUID;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Executors;
 import java.util.logging.Level;
@@ -34,17 +33,13 @@
 
 import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
 import edu.uci.ics.hyracks.api.client.IHyracksClientInterface;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.context.ICCContext;
-import edu.uci.ics.hyracks.api.control.CCConfig;
-import edu.uci.ics.hyracks.api.control.IClusterController;
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.api.control.NCConfig;
-import edu.uci.ics.hyracks.api.control.NodeParameters;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.api.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.api.job.profiling.om.StageletProfile;
 import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
 import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
@@ -56,18 +51,28 @@
 import edu.uci.ics.hyracks.control.cc.job.manager.events.JobStartEvent;
 import edu.uci.ics.hyracks.control.cc.job.manager.events.NodeHeartbeatEvent;
 import edu.uci.ics.hyracks.control.cc.job.manager.events.RegisterNodeEvent;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.RegisterPartitionAvailibilityEvent;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.RegisterPartitionRequestEvent;
 import edu.uci.ics.hyracks.control.cc.job.manager.events.RemoveDeadNodesEvent;
 import edu.uci.ics.hyracks.control.cc.job.manager.events.ReportProfilesEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.StageletCompleteEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.StageletFailureEvent;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.TaskCompleteEvent;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.TaskFailureEvent;
 import edu.uci.ics.hyracks.control.cc.job.manager.events.UnregisterNodeEvent;
 import edu.uci.ics.hyracks.control.cc.jobqueue.FutureValue;
 import edu.uci.ics.hyracks.control.cc.jobqueue.JobQueue;
-import edu.uci.ics.hyracks.control.cc.scheduler.IScheduler;
-import edu.uci.ics.hyracks.control.cc.scheduler.naive.NaiveScheduler;
 import edu.uci.ics.hyracks.control.cc.web.WebServer;
 import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
+import edu.uci.ics.hyracks.control.common.base.IClusterController;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
+import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
 
 public class ClusterControllerService extends AbstractRemoteService implements IClusterController,
         IHyracksClientInterface {
@@ -89,18 +94,20 @@
 
     private ClusterControllerInfo info;
 
-    private final Map<UUID, JobRun> runMap;
+    private final Map<JobId, JobRun> runMap;
 
     private final JobQueue jobQueue;
 
-    private final IScheduler scheduler;
-
     private final Executor taskExecutor;
 
     private final Timer timer;
 
+    private final CCClientInterface ccci;
+
     private final ICCContext ccContext;
 
+    private long jobCounter;
+
     public ClusterControllerService(CCConfig ccConfig) throws Exception {
         this.ccConfig = ccConfig;
         nodeRegistry = new LinkedHashMap<String, NodeControllerState>();
@@ -110,23 +117,24 @@
                 ClusterControllerService.class.getName()));
         taskExecutor = Executors.newCachedThreadPool();
         webServer = new WebServer(this);
-        runMap = new HashMap<UUID, JobRun>();
+        runMap = new HashMap<JobId, JobRun>();
         jobQueue = new JobQueue();
-        scheduler = new NaiveScheduler(this);
         this.timer = new Timer(true);
+        ccci = new CCClientInterface(this);
         ccContext = new ICCContext() {
             @Override
             public Map<String, Set<String>> getIPAddressNodeMap() {
                 return ipAddressNodeNameMap;
             }
         };
+        jobCounter = 0;
     }
 
     @Override
     public void start() throws Exception {
         LOGGER.log(Level.INFO, "Starting ClusterControllerService");
         Registry registry = LocateRegistry.createRegistry(ccConfig.port);
-        registry.rebind(IHyracksClientInterface.class.getName(), this);
+        registry.rebind(IHyracksClientInterface.class.getName(), ccci);
         registry.rebind(IClusterController.class.getName(), this);
         webServer.setPort(ccConfig.httpPort);
         webServer.start();
@@ -147,7 +155,7 @@
         return applications;
     }
 
-    public Map<UUID, JobRun> getRunMap() {
+    public Map<JobId, JobRun> getRunMap() {
         return runMap;
     }
 
@@ -155,10 +163,6 @@
         return jobQueue;
     }
 
-    public IScheduler getScheduler() {
-        return scheduler;
-    }
-
     public Executor getExecutor() {
         return taskExecutor;
     }
@@ -175,9 +179,13 @@
         return ccConfig;
     }
 
+    private JobId createJobId() {
+        return new JobId(jobCounter++);
+    }
+
     @Override
-    public UUID createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
-        UUID jobId = UUID.randomUUID();
+    public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+        JobId jobId = createJobId();
         JobCreateEvent jce = new JobCreateEvent(this, jobId, appName, jobSpec, jobFlags);
         jobQueue.schedule(jce);
         jce.sync();
@@ -185,10 +193,12 @@
     }
 
     @Override
-    public NodeParameters registerNode(INodeController nodeController) throws Exception {
-        String id = nodeController.getId();
-        NCConfig ncConfig = nodeController.getConfiguration();
-        NodeControllerState state = new NodeControllerState(nodeController, ncConfig);
+    public NodeParameters registerNode(NodeRegistration reg) throws Exception {
+        INodeController nodeController = reg.getNodeController();
+        String id = reg.getNodeId();
+        NCConfig ncConfig = reg.getNCConfig();
+        NetworkAddress dataPort = reg.getDataPort();
+        NodeControllerState state = new NodeControllerState(nodeController, ncConfig, dataPort);
         jobQueue.scheduleAndSync(new RegisterNodeEvent(this, id, state));
         nodeController.notifyRegistration(this);
         LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
@@ -207,33 +217,34 @@
     }
 
     @Override
-    public void notifyStageletComplete(UUID jobId, UUID stageId, int attempt, String nodeId, StageletProfile statistics)
+    public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
             throws Exception {
-        StageletCompleteEvent sce = new StageletCompleteEvent(this, jobId, stageId, attempt, nodeId);
+        TaskCompleteEvent sce = new TaskCompleteEvent(this, jobId, taskId, nodeId);
         jobQueue.schedule(sce);
     }
 
     @Override
-    public void notifyStageletFailure(UUID jobId, UUID stageId, int attempt, String nodeId) throws Exception {
-        StageletFailureEvent sfe = new StageletFailureEvent(this, jobId, stageId, attempt, nodeId);
-        jobQueue.schedule(sfe);
+    public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, Exception exception)
+            throws Exception {
+        TaskFailureEvent tfe = new TaskFailureEvent(this, jobId, taskId, nodeId, exception);
+        jobQueue.schedule(tfe);
     }
 
     @Override
-    public JobStatus getJobStatus(UUID jobId) throws Exception {
+    public JobStatus getJobStatus(JobId jobId) throws Exception {
         GetJobStatusEvent gse = new GetJobStatusEvent(this, jobId);
         jobQueue.scheduleAndSync(gse);
         return gse.getStatus();
     }
 
     @Override
-    public void start(UUID jobId) throws Exception {
+    public void start(JobId jobId) throws Exception {
         JobStartEvent jse = new JobStartEvent(this, jobId);
-        jobQueue.scheduleAndSync(jse);
+        jobQueue.schedule(jse);
     }
 
     @Override
-    public void waitForCompletion(UUID jobId) throws Exception {
+    public void waitForCompletion(JobId jobId) throws Exception {
         GetJobStatusConditionVariableEvent e = new GetJobStatusConditionVariableEvent(this, jobId);
         jobQueue.scheduleAndSync(e);
         IJobStatusConditionVariable var = e.getConditionVariable();
@@ -249,9 +260,6 @@
 
     @Override
     public synchronized void nodeHeartbeat(String id) throws Exception {
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Heartbeat from: " + id);
-        }
         jobQueue.schedule(new NodeHeartbeatEvent(this, id));
     }
 
@@ -285,6 +293,16 @@
         return info;
     }
 
+    @Override
+    public void registerPartitionProvider(PartitionDescriptor partitionDescriptor) {
+        jobQueue.schedule(new RegisterPartitionAvailibilityEvent(this, partitionDescriptor));
+    }
+
+    @Override
+    public void registerPartitionRequest(PartitionRequest partitionRequest) {
+        jobQueue.schedule(new RegisterPartitionRequestEvent(this, partitionRequest));
+    }
+
     private class DeadNodeSweeper extends TimerTask {
         @Override
         public void run() {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
index 40eb9ae..2b059b2 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
@@ -1,25 +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.control.cc;
 
 import java.util.HashSet;
 import java.util.Set;
-import java.util.UUID;
 
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.api.control.NCConfig;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 
 public class NodeControllerState {
     private final INodeController nodeController;
 
     private final NCConfig ncConfig;
 
-    private final Set<UUID> activeJobIds;
+    private final NetworkAddress dataPort;
+
+    private final Set<JobId> activeJobIds;
 
     private int lastHeartbeatDuration;
 
-    public NodeControllerState(INodeController nodeController, NCConfig ncConfig) {
+    public NodeControllerState(INodeController nodeController, NCConfig ncConfig, NetworkAddress dataPort) {
         this.nodeController = nodeController;
         this.ncConfig = ncConfig;
-        activeJobIds = new HashSet<UUID>();
+        this.dataPort = dataPort;
+        activeJobIds = new HashSet<JobId>();
     }
 
     public void notifyHeartbeat() {
@@ -42,7 +60,11 @@
         return ncConfig;
     }
 
-    public Set<UUID> getActiveJobIds() {
+    public Set<JobId> getActiveJobIds() {
         return activeJobIds;
     }
+
+    public NetworkAddress getDataPort() {
+        return dataPort;
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
index 96e29d3..c1c0161 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
@@ -1,10 +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.control.cc.application;
 
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.UUID;
 
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
 import edu.uci.ics.hyracks.api.application.ICCBootstrap;
@@ -12,6 +25,7 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.IJobLifecycleListener;
 import edu.uci.ics.hyracks.api.job.IJobSpecificationFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.control.cc.job.DeserializingJobSpecificationFactory;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
@@ -46,7 +60,7 @@
         this.jobSpecFactory = jobSpecFactory;
     }
 
-    public JobSpecification createJobSpecification(UUID jobId, byte[] bytes) throws HyracksException {
+    public JobSpecification createJobSpecification(byte[] bytes) throws HyracksException {
         return jobSpecFactory.createJobSpecification(bytes, (ICCBootstrap) bootstrap, this);
     }
 
@@ -67,19 +81,19 @@
         jobLifecycleListeners.add(jobLifecycleListener);
     }
 
-    public synchronized void notifyJobStart(UUID jobId) throws HyracksException {
+    public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
         for (IJobLifecycleListener l : jobLifecycleListeners) {
             l.notifyJobStart(jobId);
         }
     }
 
-    public synchronized void notifyJobFinish(UUID jobId) throws HyracksException {
+    public synchronized void notifyJobFinish(JobId jobId) throws HyracksException {
         for (IJobLifecycleListener l : jobLifecycleListeners) {
             l.notifyJobFinish(jobId);
         }
     }
 
-    public synchronized void notifyJobCreation(UUID jobId, JobSpecification specification) throws HyracksException {
+    public synchronized void notifyJobCreation(JobId jobId, JobSpecification specification) throws HyracksException {
         for (IJobLifecycleListener l : jobLifecycleListeners) {
             l.notifyJobCreation(jobId, specification);
         }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityCluster.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityCluster.java
new file mode 100644
index 0000000..14cebb2
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityCluster.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.job;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+
+public class ActivityCluster {
+    private final JobRun jobRun;
+
+    private final Set<ActivityId> activities;
+
+    private final Set<ActivityCluster> dependencies;
+
+    private final Set<ActivityCluster> dependents;
+
+    private ActivityClusterId id;
+
+    private ActivityClusterPlan acp;
+
+    public ActivityCluster(JobRun jobRun, Set<ActivityId> activities) {
+        this.jobRun = jobRun;
+        this.activities = activities;
+        dependencies = new HashSet<ActivityCluster>();
+        dependents = new HashSet<ActivityCluster>();
+    }
+
+    public Set<ActivityId> getActivities() {
+        return activities;
+    }
+
+    public void addDependency(ActivityCluster stage) {
+        dependencies.add(stage);
+    }
+
+    public void addDependent(ActivityCluster stage) {
+        dependents.add(stage);
+    }
+
+    public Set<ActivityCluster> getDependencies() {
+        return dependencies;
+    }
+
+    public Set<ActivityCluster> getDependents() {
+        return dependents;
+    }
+
+    public JobRun getJobRun() {
+        return jobRun;
+    }
+
+    public ActivityClusterId getActivityClusterId() {
+        return id;
+    }
+
+    public void setActivityClusterId(ActivityClusterId id) {
+        this.id = id;
+    }
+
+    public int getMaxTaskClusterAttempts() {
+        return jobRun.getJobActivityGraph().getJobSpecification().getMaxAttempts();
+    }
+
+    public ActivityClusterPlan getPlan() {
+        return acp;
+    }
+
+    public void setPlan(ActivityClusterPlan acp) {
+        this.acp = acp;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityClusterId.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityClusterId.java
new file mode 100644
index 0000000..46628a4
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityClusterId.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.control.cc.job;
+
+import java.io.Serializable;
+
+public final class ActivityClusterId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final int id;
+
+    public ActivityClusterId(int id) {
+        this.id = id;
+    }
+
+    public int getId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + id;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        ActivityClusterId other = (ActivityClusterId) obj;
+        if (id != other.id)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "AC:" + id;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityClusterPlan.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityClusterPlan.java
new file mode 100644
index 0000000..2b1d3e2
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityClusterPlan.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.control.cc.job;
+
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+
+public class ActivityClusterPlan {
+    private final Map<ActivityId, ActivityPlan> activityPlanMap;
+
+    private final TaskCluster[] taskClusters;
+
+    public ActivityClusterPlan(TaskCluster[] taskClusters, Map<ActivityId, ActivityPlan> activityPlanMap) {
+        this.activityPlanMap = activityPlanMap;
+        this.taskClusters = taskClusters;
+    }
+
+    public Map<ActivityId, ActivityPlan> getActivityPlanMap() {
+        return activityPlanMap;
+    }
+
+    public TaskCluster[] getTaskClusters() {
+        return taskClusters;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityPlan.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityPlan.java
new file mode 100644
index 0000000..beeca27
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityPlan.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.control.cc.job;
+
+import edu.uci.ics.hyracks.control.cc.scheduler.ActivityPartitionDetails;
+
+public class ActivityPlan {
+    private final ActivityPartitionDetails apd;
+
+    private Task[] tasks;
+
+    public ActivityPlan(ActivityPartitionDetails apd) {
+        this.apd = apd;
+    }
+
+    public ActivityPartitionDetails getActivityPartitionDetails() {
+        return apd;
+    }
+
+    public Task[] getTasks() {
+        return tasks;
+    }
+
+    public void setTasks(Task[] tasks) {
+        this.tasks = tasks;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobActivityGraphBuilder.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobActivityGraphBuilder.java
new file mode 100644
index 0000000..6effd84
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobActivityGraphBuilder.java
@@ -0,0 +1,92 @@
+package edu.uci.ics.hyracks.control.cc.job;
+
+import java.util.ArrayList;
+import java.util.EnumSet;
+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.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class JobActivityGraphBuilder implements IActivityGraphBuilder {
+    private static final Logger LOGGER = Logger.getLogger(JobActivityGraphBuilder.class.getName());
+
+    private JobActivityGraph jag;
+
+    @Override
+    public void addBlockingEdge(IActivity blocker, IActivity blocked) {
+        addToValueSet(jag.getBlocker2BlockedMap(), blocker.getActivityId(), blocked.getActivityId());
+        addToValueSet(jag.getBlocked2BlockerMap(), blocked.getActivityId(), blocker.getActivityId());
+    }
+
+    @Override
+    public void addSourceEdge(int operatorInputIndex, IActivity task, int taskInputIndex) {
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("Adding source edge: " + task.getActivityId().getOperatorDescriptorId() + ":"
+                    + operatorInputIndex + " -> " + task.getActivityId() + ":" + taskInputIndex);
+        }
+        insertIntoIndexedMap(jag.getActivityInputMap(), task.getActivityId(), taskInputIndex, operatorInputIndex);
+        insertIntoIndexedMap(jag.getOperatorInputMap(), task.getActivityId().getOperatorDescriptorId(),
+                operatorInputIndex, task.getActivityId());
+    }
+
+    @Override
+    public void addTargetEdge(int operatorOutputIndex, IActivity task, int taskOutputIndex) {
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("Adding target edge: " + task.getActivityId().getOperatorDescriptorId() + ":"
+                    + operatorOutputIndex + " -> " + task.getActivityId() + ":" + taskOutputIndex);
+        }
+        insertIntoIndexedMap(jag.getActivityOutputMap(), task.getActivityId(), taskOutputIndex, operatorOutputIndex);
+        insertIntoIndexedMap(jag.getOperatorOutputMap(), task.getActivityId().getOperatorDescriptorId(),
+                operatorOutputIndex, task.getActivityId());
+    }
+
+    @Override
+    public void addActivity(IActivity task) {
+        ActivityId activityId = task.getActivityId();
+        jag.getActivityNodeMap().put(activityId, task);
+        addToValueSet(jag.getOperatorActivityMap(), activityId.getOperatorDescriptorId(), activityId);
+    }
+
+    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);
+        }
+    }
+
+    public void init(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
+        jag = new JobActivityGraph(appName, 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);
+    }
+
+    public JobActivityGraph getActivityGraph() {
+        return jag;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobAttempt.java
deleted file mode 100644
index 284c92f..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobAttempt.java
+++ /dev/null
@@ -1,130 +0,0 @@
-package edu.uci.ics.hyracks.control.cc.job;
-
-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.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.control.cc.scheduler.IJobAttemptSchedulerState;
-import edu.uci.ics.hyracks.control.cc.scheduler.IScheduler;
-
-public class JobAttempt {
-    private final JobRun jobRun;
-
-    private final JobPlan plan;
-
-    private final int attempt;
-
-    private final JobStage endStage;
-
-    private final JobProfile profile;
-
-    private final Map<UUID, JobStage> stageMap;
-
-    private final Map<UUID, JobStageAttempt> stageAttemptMap;
-
-    private final Set<UUID> pendingStages;
-
-    private final Set<UUID> completedStages;
-
-    private final Set<UUID> inProgressStages;
-
-    private final Set<String> participatingNodeIds;
-
-    private final IJobAttemptSchedulerState schedulerState;
-
-    public JobAttempt(JobRun jobRun, JobPlan plan, int attempt, IScheduler scheduler) {
-        this.jobRun = jobRun;
-        this.plan = plan;
-        this.attempt = attempt;
-        this.endStage = new JobPlanner().createStageDAG(plan);
-        stageMap = new HashMap<UUID, JobStage>();
-        stageAttemptMap = new HashMap<UUID, JobStageAttempt>();
-        completedStages = new HashSet<UUID>();
-        inProgressStages = new HashSet<UUID>();
-        profile = new JobProfile(plan.getJobId(), attempt);
-        populateJobStageMap(stageMap, endStage);
-        pendingStages = new HashSet<UUID>(stageMap.keySet());
-        participatingNodeIds = new HashSet<String>();
-        schedulerState = scheduler.createJobAttemptState(this);
-    }
-
-    private static void populateJobStageMap(Map<UUID, JobStage> stageMap, JobStage stage) {
-        stageMap.put(stage.getId(), stage);
-        for (JobStage s : stage.getDependencies()) {
-            populateJobStageMap(stageMap, s);
-        }
-    }
-
-    public JobRun getJobRun() {
-        return jobRun;
-    }
-
-    public JobPlan getPlan() {
-        return plan;
-    }
-
-    public int getAttempt() {
-        return attempt;
-    }
-
-    public JobStage getEndStage() {
-        return endStage;
-    }
-
-    public void findRunnableStages(Set<JobStage> runnableStages) {
-        findRunnableStages(runnableStages, endStage);
-    }
-
-    private void findRunnableStages(Set<JobStage> runnableStages, JobStage stage) {
-        if (completedStages.contains(stage.getId()) || inProgressStages.contains(stage.getId())
-                || runnableStages.contains(stage)) {
-            return;
-        }
-        boolean runnable = true;
-        for (JobStage s : stage.getDependencies()) {
-            if (!completedStages.contains(s.getId())) {
-                runnable = false;
-                findRunnableStages(runnableStages, s);
-            }
-        }
-        if (runnable) {
-            runnableStages.add(stage);
-        }
-    }
-
-    public Set<UUID> getPendingStageIds() {
-        return pendingStages;
-    }
-
-    public Set<UUID> getInProgressStageIds() {
-        return inProgressStages;
-    }
-
-    public Set<UUID> getCompletedStageIds() {
-        return completedStages;
-    }
-
-    public Map<UUID, JobStage> getStageMap() {
-        return stageMap;
-    }
-
-    public Map<UUID, JobStageAttempt> getStageAttemptMap() {
-        return stageAttemptMap;
-    }
-
-    public Set<String> getParticipatingNodeIds() {
-        return participatingNodeIds;
-    }
-
-    public JobProfile getJobProfile() {
-        return profile;
-    }
-
-    public IJobAttemptSchedulerState getSchedulerState() {
-        return schedulerState;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanBuilder.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanBuilder.java
deleted file mode 100644
index bafa7e5..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanBuilder.java
+++ /dev/null
@@ -1,91 +0,0 @@
-package edu.uci.ics.hyracks.control.cc.job;
-
-import java.util.ArrayList;
-import java.util.EnumSet;
-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.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
-import edu.uci.ics.hyracks.api.job.JobFlag;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-
-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);
-        }
-    }
-
-    public void init(String appName, UUID jobId, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
-        plan = new JobPlan(appName, jobId, 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);
-    }
-
-    public JobPlan getPlan() {
-        return plan;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanner.java
deleted file mode 100644
index 900e5eee..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanner.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.job;
-
-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.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.JobPlan;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.api.util.Pair;
-
-public class JobPlanner {
-    private static final Logger LOGGER = Logger.getLogger(JobPlanner.class.getName());
-
-    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(JobPlan plan) {
-        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;
-    }
-
-    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 JobStage createStageDAG(JobPlan jag) {
-        JobStage endStage = inferStages(jag);
-        return endStage;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
index 5766288..6231080 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
@@ -14,53 +14,86 @@
  */
 package edu.uci.ics.hyracks.control.cc.job;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.control.cc.scheduler.IScheduler;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
+import edu.uci.ics.hyracks.control.cc.scheduler.ActivityPartitionDetails;
+import edu.uci.ics.hyracks.control.cc.scheduler.JobScheduler;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
 
 public class JobRun implements IJobStatusConditionVariable {
-    private final JobPlan plan;
-    private final List<JobAttempt> attempts;
+    private final JobId jobId;
+
+    private final JobActivityGraph jag;
+
+    private final PartitionMatchMaker pmm;
+
+    private final Set<String> participatingNodeIds;
+
+    private final JobProfile profile;
+
+    private Set<ActivityCluster> activityClusters;
+
+    private final Map<ActivityId, ActivityCluster> activityClusterMap;
+
+    private final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicyMap;
+
+    private JobScheduler js;
+
     private JobStatus status;
-    private Set<ConstraintExpression> constraints;
 
-    public JobRun(JobPlan plan, Set<ConstraintExpression> constraints) {
-        this.plan = plan;
-        attempts = new ArrayList<JobAttempt>();
-        this.constraints = constraints;
+    private Exception exception;
+
+    public JobRun(JobId jobId, JobActivityGraph plan) {
+        this.jobId = jobId;
+        this.jag = plan;
+        pmm = new PartitionMatchMaker();
+        participatingNodeIds = new HashSet<String>();
+        profile = new JobProfile(jobId);
+        activityClusterMap = new HashMap<ActivityId, ActivityCluster>();
+        connectorPolicyMap = new HashMap<ConnectorDescriptorId, IConnectorPolicy>();
     }
 
-    public JobPlan getJobPlan() {
-        return plan;
+    public JobId getJobId() {
+        return jobId;
     }
 
-    public synchronized void setStatus(JobStatus status) {
+    public JobActivityGraph getJobActivityGraph() {
+        return jag;
+    }
+
+    public PartitionMatchMaker getPartitionMatchMaker() {
+        return pmm;
+    }
+
+    public synchronized void setStatus(JobStatus status, Exception exception) {
         this.status = status;
+        this.exception = exception;
         notifyAll();
     }
 
-    public JobStatus getStatus() {
+    public synchronized JobStatus getStatus() {
         return status;
     }
 
-    public List<JobAttempt> getAttempts() {
-        return attempts;
-    }
-
-    public Set<ConstraintExpression> getConstraints() {
-        return constraints;
-    }
-
-    public JobAttempt createAttempt(IScheduler scheduler) {
-        int attemptNumber = attempts.size();
-        JobAttempt attempt = new JobAttempt(this, plan, attemptNumber, scheduler);
-        attempts.add(attempt);
-        return attempt;
+    public synchronized Exception getException() {
+        return exception;
     }
 
     @Override
@@ -68,5 +101,164 @@
         while (status != JobStatus.TERMINATED && status != JobStatus.FAILURE) {
             wait();
         }
+        if (exception != null) {
+            throw new HyracksException("Job Failed", exception);
+        }
+    }
+
+    public Set<String> getParticipatingNodeIds() {
+        return participatingNodeIds;
+    }
+
+    public JobProfile getJobProfile() {
+        return profile;
+    }
+
+    public void setScheduler(JobScheduler js) {
+        this.js = js;
+    }
+
+    public JobScheduler getScheduler() {
+        return js;
+    }
+
+    public Map<ActivityId, ActivityCluster> getActivityClusterMap() {
+        return activityClusterMap;
+    }
+
+    public Set<ActivityCluster> getActivityClusters() {
+        return activityClusters;
+    }
+
+    public void setActivityClusters(Set<ActivityCluster> activityClusters) {
+        this.activityClusters = activityClusters;
+    }
+
+    public Map<ConnectorDescriptorId, IConnectorPolicy> getConnectorPolicyMap() {
+        return connectorPolicyMap;
+    }
+
+    public JSONObject toJSON() throws JSONException {
+        JSONObject result = new JSONObject();
+
+        result.put("job-id", jobId.toString());
+
+        JSONArray aClusters = new JSONArray();
+        for (ActivityCluster ac : activityClusters) {
+            JSONObject acJSON = new JSONObject();
+
+            acJSON.put("activity-cluster-id", String.valueOf(ac.getActivityClusterId()));
+
+            JSONArray activitiesJSON = new JSONArray();
+            for (ActivityId aid : ac.getActivities()) {
+                activitiesJSON.put(aid);
+            }
+            acJSON.put("activities", activitiesJSON);
+
+            JSONArray dependentsJSON = new JSONArray();
+            for (ActivityCluster dependent : ac.getDependents()) {
+                dependentsJSON.put(String.valueOf(dependent.getActivityClusterId()));
+            }
+            acJSON.put("dependents", dependentsJSON);
+
+            JSONArray dependenciesJSON = new JSONArray();
+            for (ActivityCluster dependency : ac.getDependencies()) {
+                dependenciesJSON.put(String.valueOf(dependency.getActivityClusterId()));
+            }
+            acJSON.put("dependencies", dependenciesJSON);
+
+            ActivityClusterPlan acp = ac.getPlan();
+            if (acp == null) {
+                acJSON.put("plan", (Object) null);
+            } else {
+                JSONObject planJSON = new JSONObject();
+
+                JSONArray acTasks = new JSONArray();
+                for (Map.Entry<ActivityId, ActivityPlan> e : acp.getActivityPlanMap().entrySet()) {
+                    ActivityPlan acPlan = e.getValue();
+                    JSONObject entry = new JSONObject();
+                    entry.put("activity-id", e.getKey().toString());
+
+                    ActivityPartitionDetails apd = acPlan.getActivityPartitionDetails();
+                    entry.put("partition-count", apd.getPartitionCount());
+
+                    JSONArray inPartCountsJSON = new JSONArray();
+                    int[] inPartCounts = apd.getInputPartitionCounts();
+                    if (inPartCounts != null) {
+                        for (int i : inPartCounts) {
+                            inPartCountsJSON.put(i);
+                        }
+                    }
+                    entry.put("input-partition-counts", inPartCountsJSON);
+
+                    JSONArray outPartCountsJSON = new JSONArray();
+                    int[] outPartCounts = apd.getOutputPartitionCounts();
+                    if (outPartCounts != null) {
+                        for (int o : outPartCounts) {
+                            outPartCountsJSON.put(o);
+                        }
+                    }
+                    entry.put("output-partition-counts", outPartCountsJSON);
+
+                    JSONArray tasks = new JSONArray();
+                    for (Task t : acPlan.getTasks()) {
+                        JSONObject task = new JSONObject();
+
+                        task.put("task-id", t.getTaskId().toString());
+
+                        JSONArray dependentTasksJSON = new JSONArray();
+                        for (TaskId dependent : t.getDependents()) {
+                            dependentTasksJSON.put(dependent.toString());
+                        }
+                        task.put("dependents", dependentTasksJSON);
+
+                        JSONArray dependencyTasksJSON = new JSONArray();
+                        for (TaskId dependency : t.getDependencies()) {
+                            dependencyTasksJSON.put(dependency.toString());
+                        }
+                        task.put("dependencies", dependencyTasksJSON);
+
+                        tasks.put(task);
+                    }
+                    entry.put("tasks", tasks);
+
+                    acTasks.put(entry);
+                }
+                planJSON.put("task-map", acTasks);
+
+                JSONArray tClusters = new JSONArray();
+                for (TaskCluster tc : acp.getTaskClusters()) {
+                    JSONObject c = new JSONObject();
+                    c.put("task-cluster-id", String.valueOf(tc.getTaskClusterId()));
+
+                    JSONArray tasks = new JSONArray();
+                    for (Task t : tc.getTasks()) {
+                        tasks.put(t.getTaskId().toString());
+                    }
+                    c.put("tasks", tasks);
+
+                    JSONArray prodParts = new JSONArray();
+                    for (PartitionId p : tc.getProducedPartitions()) {
+                        prodParts.put(p.toString());
+                    }
+                    c.put("produced-partitions", prodParts);
+
+                    JSONArray reqdParts = new JSONArray();
+                    for (PartitionId p : tc.getRequiredPartitions()) {
+                        reqdParts.put(p.toString());
+                    }
+                    c.put("required-partitions", reqdParts);
+
+                    tClusters.put(c);
+                }
+                planJSON.put("task-clusters", tClusters);
+
+                acJSON.put("plan", planJSON);
+            }
+            aClusters.put(acJSON);
+        }
+        result.put("activity-clusters", aClusters);
+
+        return result;
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStage.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStage.java
deleted file mode 100644
index 8e4013d..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStage.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.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-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStageAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStageAttempt.java
deleted file mode 100644
index 61db770..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStageAttempt.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.job;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.control.cc.scheduler.ISchedule;
-
-public class JobStageAttempt {
-    private final JobStage stage;
-
-    private final JobAttempt jobAttempt;
-
-    private final Set<String> participatingNodes;
-
-    private final Set<String> completedNodes;
-
-    private ISchedule schedule;
-
-    public JobStageAttempt(JobStage stage, JobAttempt jobAttempt) {
-        this.stage = stage;
-        this.jobAttempt = jobAttempt;
-        participatingNodes = new HashSet<String>();
-        completedNodes = new HashSet<String>();
-    }
-
-    public JobStage getJobStage() {
-        return stage;
-    }
-
-    public JobAttempt getJobAttempt() {
-        return jobAttempt;
-    }
-
-    public void setSchedule(ISchedule schedule) {
-        this.schedule = schedule;
-        for (ActivityNodeId aid : stage.getTasks()) {
-            String[] partitions = schedule.getPartitions(aid);
-            for (String nid : partitions) {
-                participatingNodes.add(nid);
-            }
-        }
-    }
-
-    public ISchedule getSchedule() {
-        return schedule;
-    }
-
-    public Set<String> getParticipatingNodes() {
-        return participatingNodes;
-    }
-
-    public Set<String> getCompletedNodes() {
-        return completedNodes;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/Task.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/Task.java
new file mode 100644
index 0000000..84ae6e5
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/Task.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.control.cc.job;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+
+public class Task {
+    private final TaskId taskId;
+
+    private final ActivityPlan activityPlan;
+
+    private final Set<TaskId> dependencies;
+
+    private final Set<TaskId> dependents;
+
+    private TaskCluster taskCluster;
+
+    public Task(TaskId taskId, ActivityPlan activityPlan) {
+        this.taskId = taskId;
+        this.activityPlan = activityPlan;
+        this.dependencies = new HashSet<TaskId>();
+        this.dependents = new HashSet<TaskId>();
+    }
+
+    public TaskId getTaskId() {
+        return taskId;
+    }
+
+    public ActivityPlan getActivityPlan() {
+        return activityPlan;
+    }
+
+    public Set<TaskId> getDependencies() {
+        return dependencies;
+    }
+
+    public Set<TaskId> getDependents() {
+        return dependents;
+    }
+
+    public TaskCluster getTaskCluster() {
+        return taskCluster;
+    }
+
+    public void setTaskCluster(TaskCluster taskCluster) {
+        this.taskCluster = taskCluster;
+    }
+
+    @Override
+    public String toString() {
+        return String.valueOf(taskId);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
new file mode 100644
index 0000000..22f5587
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.job;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+
+public class TaskAttempt {
+    public enum TaskStatus {
+        INITIALIZED,
+        RUNNING,
+        COMPLETED,
+        FAILED,
+        ABORTED,
+    }
+
+    private final TaskClusterAttempt tcAttempt;
+
+    private final TaskAttemptId taskId;
+
+    private final Task taskState;
+
+    private String nodeId;
+
+    private TaskStatus status;
+
+    private Exception exception;
+
+    public TaskAttempt(TaskClusterAttempt tcAttempt, TaskAttemptId taskId, Task taskState) {
+        this.tcAttempt = tcAttempt;
+        this.taskId = taskId;
+        this.taskState = taskState;
+    }
+
+    public TaskClusterAttempt getTaskClusterAttempt() {
+        return tcAttempt;
+    }
+
+    public TaskAttemptId getTaskAttemptId() {
+        return taskId;
+    }
+
+    public Task getTaskState() {
+        return taskState;
+    }
+
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    public TaskStatus getStatus() {
+        return status;
+    }
+
+    public Exception getException() {
+        return exception;
+    }
+
+    public void setStatus(TaskStatus status, Exception exception) {
+        this.status = status;
+        this.exception = exception;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskCluster.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskCluster.java
new file mode 100644
index 0000000..f41cda9
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskCluster.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.control.cc.job;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class TaskCluster {
+    private final TaskClusterId taskClusterId;
+
+    private final ActivityCluster ac;
+
+    private final Task[] tasks;
+
+    private final Set<PartitionId> producedPartitions;
+
+    private final Set<PartitionId> requiredPartitions;
+
+    private final Set<TaskCluster> dependencyTaskClusters;
+
+    private final Set<TaskCluster> dependentTaskClusters;
+
+    private final List<TaskClusterAttempt> taskClusterAttempts;
+
+    public TaskCluster(TaskClusterId taskClusterId, ActivityCluster ac, Task[] tasks) {
+        this.taskClusterId = taskClusterId;
+        this.ac = ac;
+        this.tasks = tasks;
+        producedPartitions = new HashSet<PartitionId>();
+        requiredPartitions = new HashSet<PartitionId>();
+        dependencyTaskClusters = new HashSet<TaskCluster>();
+        dependentTaskClusters = new HashSet<TaskCluster>();
+        taskClusterAttempts = new ArrayList<TaskClusterAttempt>();
+    }
+
+    public TaskClusterId getTaskClusterId() {
+        return taskClusterId;
+    }
+
+    public ActivityCluster getActivityCluster() {
+        return ac;
+    }
+
+    public Task[] getTasks() {
+        return tasks;
+    }
+
+    public Set<PartitionId> getProducedPartitions() {
+        return producedPartitions;
+    }
+
+    public Set<PartitionId> getRequiredPartitions() {
+        return requiredPartitions;
+    }
+
+    public Set<TaskCluster> getDependencyTaskClusters() {
+        return dependencyTaskClusters;
+    }
+
+    public Set<TaskCluster> getDependentTaskClusters() {
+        return dependentTaskClusters;
+    }
+
+    public List<TaskClusterAttempt> getAttempts() {
+        return taskClusterAttempts;
+    }
+
+    @Override
+    public String toString() {
+        return "TC:" + Arrays.toString(tasks);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java
new file mode 100644
index 0000000..9d33d12
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.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.control.cc.job;
+
+public class TaskClusterAttempt {
+    public enum TaskClusterStatus {
+        RUNNING,
+        COMPLETED,
+        FAILED,
+        ABORTED,
+    }
+
+    private final TaskCluster taskCluster;
+
+    private final int attempt;
+
+    private TaskAttempt[] taskAttempts;
+
+    private TaskClusterStatus status;
+
+    private int pendingTaskCounter;
+
+    public TaskClusterAttempt(TaskCluster taskCluster, int attempt) {
+        this.taskCluster = taskCluster;
+        this.attempt = attempt;
+    }
+
+    public TaskCluster getTaskCluster() {
+        return taskCluster;
+    }
+
+    public void setTaskAttempts(TaskAttempt[] taskAttempts) {
+        this.taskAttempts = taskAttempts;
+    }
+
+    public TaskAttempt[] getTaskAttempts() {
+        return taskAttempts;
+    }
+
+    public int getAttempt() {
+        return attempt;
+    }
+
+    public void setStatus(TaskClusterStatus status) {
+        this.status = status;
+    }
+
+    public TaskClusterStatus getStatus() {
+        return status;
+    }
+
+    public void initializePendingTaskCounter() {
+        pendingTaskCounter = taskAttempts.length;
+    }
+
+    public int getPendingTaskCounter() {
+        return pendingTaskCounter;
+    }
+
+    public int decrementPendingTasksCounter() {
+        return --pendingTaskCounter;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterId.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterId.java
new file mode 100644
index 0000000..b1000dd
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterId.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.control.cc.job;
+
+import java.io.Serializable;
+
+public final class TaskClusterId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final ActivityClusterId activityClusterId;
+
+    private final int id;
+
+    public TaskClusterId(ActivityClusterId activityClusterId, int id) {
+        this.activityClusterId = activityClusterId;
+        this.id = id;
+    }
+
+    public ActivityClusterId getActivityClusterId() {
+        return activityClusterId;
+    }
+
+    public int getId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((activityClusterId == null) ? 0 : activityClusterId.hashCode());
+        result = prime * result + id;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        TaskClusterId other = (TaskClusterId) obj;
+        if (activityClusterId == null) {
+            if (other.activityClusterId != null)
+                return false;
+        } else if (!activityClusterId.equals(other.activityClusterId))
+            return false;
+        if (id != other.id)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "TC:" + activityClusterId + ":" + id;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManager.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManager.java
deleted file mode 100644
index ef87c4f..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManager.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.job.manager;
-
-import java.util.EnumSet;
-import java.util.Map;
-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;
-
-public interface IJobManager {
-    public void createJob(UUID jobId, String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags)
-            throws Exception;
-
-    public void start(UUID jobId) throws Exception;
-
-    public void notifyStageletComplete(UUID jobId, UUID stageId, int attempt, String nodeId,
-            Map<String, Long> statistics) throws Exception;
-
-    public void notifyStageletFailure(UUID jobId, UUID stageId, int attempt, String nodeId) throws Exception;
-
-    public JobStatus getJobStatus(UUID jobId);
-
-    public void waitForCompletion(UUID jobId) throws Exception;
-
-    public void notifyNodeFailure(String nodeId) throws Exception;
-
-    public void registerNode(String nodeId) throws Exception;
-
-    public void reportProfile(String id, Map<UUID, Map<String, Long>> counterDump) throws Exception;
-
-    public IJobManagerQueryInterface getQueryInterface();
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManagerQueryInterface.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManagerQueryInterface.java
deleted file mode 100644
index fbdf941..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManagerQueryInterface.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.job.manager;
-
-import java.util.UUID;
-
-import org.json.JSONArray;
-import org.json.JSONObject;
-
-public interface IJobManagerQueryInterface {
-    public JSONArray getAllJobSummaries() throws Exception;
-
-    public JSONObject getJobSpecification(UUID jobId) throws Exception;
-
-    public JSONObject getJobPlan(UUID jobId) throws Exception;
-
-    public JSONObject getJobProfile(UUID jobId, int attempt) throws Exception;
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/JobLifecycleHelper.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/JobLifecycleHelper.java
deleted file mode 100644
index be42e22..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/JobLifecycleHelper.java
+++ /dev/null
@@ -1,27 +0,0 @@
-package edu.uci.ics.hyracks.control.cc.job.manager;
-
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.JobAttemptStartEvent;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
-import edu.uci.ics.hyracks.control.cc.remote.ops.JobletAborter;
-
-public class JobLifecycleHelper {
-    public static void abortJob(ClusterControllerService ccs, UUID jobId, int attempt, Set<String> targetNodes) {
-        if (!targetNodes.isEmpty()) {
-            JobletAborter[] jas = new JobletAborter[targetNodes.size()];
-            int i = 0;
-            for (String nodeId : targetNodes) {
-                jas[i++] = new JobletAborter(nodeId, jobId, attempt);
-            }
-            try {
-                RemoteRunner.runRemote(ccs, jas, null);
-            } catch (Exception e) {
-                e.printStackTrace();
-            }
-        }
-        ccs.getJobQueue().schedule(new JobAttemptStartEvent(ccs, jobId));
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/AbstractTaskLifecycleEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/AbstractTaskLifecycleEvent.java
new file mode 100644
index 0000000..0028b6f
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/AbstractTaskLifecycleEvent.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.control.cc.job.manager.events;
+
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.ActivityPlan;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.job.Task;
+import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
+import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
+import edu.uci.ics.hyracks.control.cc.job.TaskClusterAttempt;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
+
+public abstract class AbstractTaskLifecycleEvent extends AbstractEvent {
+    protected final ClusterControllerService ccs;
+    protected final JobId jobId;
+    protected final TaskAttemptId taId;
+    protected final String nodeId;
+
+    public AbstractTaskLifecycleEvent(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.taId = taId;
+        this.nodeId = nodeId;
+    }
+
+    @Override
+    public final void run() {
+        JobRun run = ccs.getRunMap().get(jobId);
+        if (run != null) {
+            TaskId tid = taId.getTaskId();
+            Map<ActivityId, ActivityCluster> activityClusterMap = run.getActivityClusterMap();
+            ActivityCluster ac = activityClusterMap.get(tid.getActivityId());
+            if (ac != null) {
+                Map<ActivityId, ActivityPlan> taskStateMap = ac.getPlan().getActivityPlanMap();
+                Task[] taskStates = taskStateMap.get(tid.getActivityId()).getTasks();
+                if (taskStates != null && taskStates.length > tid.getPartition()) {
+                    Task ts = taskStates[tid.getPartition()];
+                    TaskCluster tc = ts.getTaskCluster();
+                    List<TaskClusterAttempt> taskClusterAttempts = tc.getAttempts();
+                    if (taskClusterAttempts != null && taskClusterAttempts.size() > taId.getAttempt()) {
+                        TaskClusterAttempt tca = taskClusterAttempts.get(taId.getAttempt());
+                        for (TaskAttempt ta : tca.getTaskAttempts()) {
+                            if (ta.getTaskAttemptId().equals(taId)) {
+                                performEvent(ta);
+                                break;
+                            }
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    protected abstract void performEvent(TaskAttempt ta);
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java
index 69c244a..050c6ed 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java
@@ -19,13 +19,14 @@
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
 import edu.uci.ics.hyracks.control.cc.jobqueue.FutureValue;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
 import edu.uci.ics.hyracks.control.cc.remote.ops.ApplicationDestroyer;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
 
-public class ApplicationDestroyEvent implements Runnable {
+public class ApplicationDestroyEvent extends AbstractEvent {
     private final ClusterControllerService ccs;
     private final String appName;
     private FutureValue fv;
@@ -57,7 +58,7 @@
                     fv.setException(e);
                     return;
                 }
-                ccs.getJobQueue().schedule(new Runnable() {
+                ccs.getJobQueue().schedule(new AbstractEvent() {
                     @Override
                     public void run() {
                         try {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationStartEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationStartEvent.java
index 9f4ad8f..ab05d0f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationStartEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationStartEvent.java
@@ -20,13 +20,14 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
 import edu.uci.ics.hyracks.control.cc.jobqueue.FutureValue;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
 import edu.uci.ics.hyracks.control.cc.remote.ops.ApplicationStarter;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
 
-public class ApplicationStartEvent implements Runnable {
+public class ApplicationStartEvent extends AbstractEvent {
     private final ClusterControllerService ccs;
     private final String appName;
     private final FutureValue fv;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobActivityGraphJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobActivityGraphJSONEvent.java
new file mode 100644
index 0000000..14b56df
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobActivityGraphJSONEvent.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.job.manager.events;
+
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+
+public class GetJobActivityGraphJSONEvent extends SynchronizableEvent {
+    private final ClusterControllerService ccs;
+    private final JobId jobId;
+    private JSONObject json;
+
+    public GetJobActivityGraphJSONEvent(ClusterControllerService ccs, JobId jobId) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        JobRun run = ccs.getRunMap().get(jobId);
+        if (run == null) {
+            json = new JSONObject();
+            return;
+        }
+        json = run.getJobActivityGraph().toJSON();
+    }
+
+    public JSONObject getJSON() {
+        return json;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java
index dc397ef..4355962 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java
@@ -14,44 +14,35 @@
  */
 package edu.uci.ics.hyracks.control.cc.job.manager.events;
 
-import java.util.List;
-import java.util.UUID;
-
 import org.json.JSONObject;
 
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 
-public class GetJobProfileJSONEvent extends SynchronizableRunnable {
+public class GetJobProfileJSONEvent extends SynchronizableEvent {
     private final ClusterControllerService ccs;
-    private final UUID jobId;
-    private final int attempt;
-    private JSONObject profile;
+    private final JobId jobId;
+    private JSONObject json;
 
-    public GetJobProfileJSONEvent(ClusterControllerService ccs, UUID jobId, int attempt) {
+    public GetJobProfileJSONEvent(ClusterControllerService ccs, JobId jobId) {
         this.ccs = ccs;
         this.jobId = jobId;
-        this.attempt = attempt;
     }
 
     @Override
     protected void doRun() throws Exception {
-        JobRun run = ccs.getRunMap().get(jobId);
-        profile = new JSONObject();
-        if (run == null) {
+        json = new JSONObject();
+        JobRun jobRun = ccs.getRunMap().get(jobId);
+        if (jobRun == null) {
+            json = new JSONObject();
             return;
         }
-        List<JobAttempt> attempts = run.getAttempts();
-        if (attempts.size() <= attempt) {
-            return;
-        }
-        JobAttempt ja = attempts.get(attempt);
-        profile = ja.getJobProfile().toJSON();
+        json = jobRun.getJobProfile().toJSON();
     }
 
-    public JSONObject getProfile() {
-        return profile;
+    public JSONObject getJSON() {
+        return json;
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSpecificationJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSpecificationJSONEvent.java
index 6248925..db7e5a7 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSpecificationJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSpecificationJSONEvent.java
@@ -14,20 +14,19 @@
  */
 package edu.uci.ics.hyracks.control.cc.job.manager.events;
 
-import java.util.UUID;
-
 import org.json.JSONObject;
 
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 
-public class GetJobSpecificationJSONEvent extends SynchronizableRunnable {
+public class GetJobSpecificationJSONEvent extends SynchronizableEvent {
     private final ClusterControllerService ccs;
-    private final UUID jobId;
-    private JSONObject spec;
+    private final JobId jobId;
+    private JSONObject json;
 
-    public GetJobSpecificationJSONEvent(ClusterControllerService ccs, UUID jobId) {
+    public GetJobSpecificationJSONEvent(ClusterControllerService ccs, JobId jobId) {
         this.ccs = ccs;
         this.jobId = jobId;
     }
@@ -36,13 +35,13 @@
     protected void doRun() throws Exception {
         JobRun run = ccs.getRunMap().get(jobId);
         if (run == null) {
-            spec = new JSONObject();
+            json = new JSONObject();
             return;
         }
-        spec = run.getJobPlan().getJobSpecification().toJSON();
+        json = run.getJobActivityGraph().getJobSpecification().toJSON();
     }
 
-    public JSONObject getSpecification() {
-        return spec;
+    public JSONObject getJSON() {
+        return json;
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusConditionVariableEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusConditionVariableEvent.java
index c80974a..357d62b 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusConditionVariableEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusConditionVariableEvent.java
@@ -14,18 +14,17 @@
  */
 package edu.uci.ics.hyracks.control.cc.job.manager.events;
 
-import java.util.UUID;
-
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 
-public class GetJobStatusConditionVariableEvent extends SynchronizableRunnable {
+public class GetJobStatusConditionVariableEvent extends SynchronizableEvent {
     private final ClusterControllerService ccs;
-    private final UUID jobId;
+    private final JobId jobId;
     private IJobStatusConditionVariable cVar;
 
-    public GetJobStatusConditionVariableEvent(ClusterControllerService ccs, UUID jobId) {
+    public GetJobStatusConditionVariableEvent(ClusterControllerService ccs, JobId jobId) {
         this.ccs = ccs;
         this.jobId = jobId;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusEvent.java
index 9b15e5c..fbe6e4c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusEvent.java
@@ -14,19 +14,18 @@
  */
 package edu.uci.ics.hyracks.control.cc.job.manager.events;
 
-import java.util.UUID;
-
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 
-public class GetJobStatusEvent extends SynchronizableRunnable {
+public class GetJobStatusEvent extends SynchronizableEvent {
     private final ClusterControllerService ccs;
-    private final UUID jobId;
+    private final JobId jobId;
     private JobStatus status;
 
-    public GetJobStatusEvent(ClusterControllerService ccs, UUID jobId) {
+    public GetJobStatusEvent(ClusterControllerService ccs, JobId jobId) {
         this.ccs = ccs;
         this.jobId = jobId;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSummariesJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSummariesJSONEvent.java
index 74851e5..bf5e5c3 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSummariesJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSummariesJSONEvent.java
@@ -19,9 +19,9 @@
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 
-public class GetJobSummariesJSONEvent extends SynchronizableRunnable {
+public class GetJobSummariesJSONEvent extends SynchronizableEvent {
     private final ClusterControllerService ccs;
     private JSONArray summaries;
 
@@ -35,8 +35,7 @@
         for (JobRun run : ccs.getRunMap().values()) {
             JSONObject jo = new JSONObject();
             jo.put("type", "job-summary");
-            jo.put("id", run.getJobPlan().getJobId().toString());
-            jo.put("attempts", run.getAttempts().size());
+            jo.put("id", run.getJobId().toString());
             jo.put("status", run.getStatus().toString());
             summaries.put(jo);
         }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeEvent.java
index 7554e62..0db7f2a 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeEvent.java
@@ -16,9 +16,9 @@
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 
-public class GetNodeEvent extends SynchronizableRunnable {
+public class GetNodeEvent extends SynchronizableEvent {
     private final ClusterControllerService ccs;
     private final String nodeId;
     private NodeControllerState state;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAbortEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAbortEvent.java
deleted file mode 100644
index 9a0eff1..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAbortEvent.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
-
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.job.manager.JobLifecycleHelper;
-
-public class JobAbortEvent implements Runnable {
-    private final ClusterControllerService ccs;
-
-    private final UUID jobId;
-
-    private final int attempt;
-
-    public JobAbortEvent(ClusterControllerService ccs, UUID jobId, int attempt) {
-        this.ccs = ccs;
-        this.jobId = jobId;
-        this.attempt = attempt;
-    }
-
-    @Override
-    public void run() {
-        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
-        Map<UUID, JobRun> runMap = ccs.getRunMap();
-        final JobRun run = runMap.get(jobId);
-        final Set<String> targetNodes = new HashSet<String>();
-        if (run != null) {
-            List<JobAttempt> jas = run.getAttempts();
-            JobAttempt ja = jas.get(attempt);
-            for (String runningNodeId : ja.getParticipatingNodeIds()) {
-                if (nodeMap.containsKey(runningNodeId)) {
-                    targetNodes.add(runningNodeId);
-                    nodeMap.get(runningNodeId).getActiveJobIds().remove(jobId);
-                }
-            }
-        }
-
-        ccs.getExecutor().execute(new Runnable() {
-            @Override
-            public void run() {
-                JobLifecycleHelper.abortJob(ccs, jobId, attempt, targetNodes);
-            }
-        });
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAttemptStartEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAttemptStartEvent.java
deleted file mode 100644
index 7dc6496..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAttemptStartEvent.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
-
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
-
-public class JobAttemptStartEvent implements Runnable {
-    private ClusterControllerService ccs;
-    private UUID jobId;
-
-    public JobAttemptStartEvent(ClusterControllerService ccs, UUID jobId) {
-        this.ccs = ccs;
-        this.jobId = jobId;
-    }
-
-    @Override
-    public void run() {
-        JobRun run = ccs.getRunMap().get(jobId);
-        int maxAttempts = run.getJobPlan().getJobSpecification().getMaxAttempts();
-        if (maxAttempts == 0) {
-            maxAttempts = ccs.getConfig().defaultMaxJobAttempts;
-        }
-        if (run.getAttempts().size() > maxAttempts) {
-            run.setStatus(JobStatus.FAILURE);
-            return;
-        }
-        JobAttempt attempt = run.createAttempt(ccs.getScheduler());
-        new ScheduleRunnableStagesEvent(ccs, jobId, attempt.getAttempt()).run();
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCleanupEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCleanupEvent.java
index c4ed077..4274914 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCleanupEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCleanupEvent.java
@@ -14,42 +14,44 @@
  */
 package edu.uci.ics.hyracks.control.cc.job.manager.events;
 
-import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
 import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
 import edu.uci.ics.hyracks.control.cc.remote.ops.JobCompleteNotifier;
 
-public class JobCleanupEvent implements Runnable {
+public class JobCleanupEvent extends AbstractEvent {
     private ClusterControllerService ccs;
-    private UUID jobId;
-    private int attempt;
+    private JobId jobId;
     private JobStatus status;
+    private Exception exception;
 
-    public JobCleanupEvent(ClusterControllerService ccs, UUID jobId, int attempt, JobStatus status) {
+    public JobCleanupEvent(ClusterControllerService ccs, JobId jobId, JobStatus status, Exception exception) {
         this.ccs = ccs;
         this.jobId = jobId;
-        this.attempt = attempt;
         this.status = status;
+        this.exception = exception;
     }
 
     @Override
     public void run() {
         final JobRun run = ccs.getRunMap().get(jobId);
-        final JobAttempt ja = run.getAttempts().get(attempt);
-        Set<String> targetNodes = ja.getParticipatingNodeIds();
+        Set<String> targetNodes = run.getParticipatingNodeIds();
         final JobCompleteNotifier[] jcns = new JobCompleteNotifier[targetNodes.size()];
         int i = 0;
         for (String n : targetNodes) {
             jcns[i++] = new JobCompleteNotifier(n, jobId);
+            NodeControllerState ncs = ccs.getNodeMap().get(n);
+            if (ncs != null) {
+                ncs.getActiveJobIds().remove(jobId);
+            }
         }
         ccs.getExecutor().execute(new Runnable() {
             @Override
@@ -61,10 +63,11 @@
                         e.printStackTrace();
                     }
                 }
-                ccs.getJobQueue().schedule(new Runnable() {
+                ccs.getJobQueue().schedule(new AbstractEvent() {
                     @Override
                     public void run() {
-                        CCApplicationContext appCtx = ccs.getApplicationMap().get(ja.getPlan().getApplicationName());
+                        CCApplicationContext appCtx = ccs.getApplicationMap().get(
+                                run.getJobActivityGraph().getApplicationName());
                         if (appCtx != null) {
                             try {
                                 appCtx.notifyJobFinish(jobId);
@@ -72,12 +75,7 @@
                                 e.printStackTrace();
                             }
                         }
-                        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
-                        for (String nodeId : ja.getParticipatingNodeIds()) {
-                            NodeControllerState state = nodeMap.get(nodeId);
-                            state.getActiveJobIds().remove(jobId);
-                        }
-                        run.setStatus(status);
+                        run.setStatus(status, exception);
                     }
                 });
             }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCreateEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCreateEvent.java
index d7f244e..5fadb00 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCreateEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCreateEvent.java
@@ -15,36 +15,31 @@
 package edu.uci.ics.hyracks.control.cc.job.manager.events;
 
 import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.UUID;
 
-import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
 import edu.uci.ics.hyracks.api.job.JobFlag;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.cc.job.IConnectorDescriptorVisitor;
 import edu.uci.ics.hyracks.control.cc.job.IOperatorDescriptorVisitor;
-import edu.uci.ics.hyracks.control.cc.job.JobPlanBuilder;
+import edu.uci.ics.hyracks.control.cc.job.JobActivityGraphBuilder;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
 import edu.uci.ics.hyracks.control.cc.job.PlanUtils;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.cc.scheduler.JobScheduler;
 
-public class JobCreateEvent extends SynchronizableRunnable {
+public class JobCreateEvent extends SynchronizableEvent {
     private final ClusterControllerService ccs;
     private final byte[] jobSpec;
     private final EnumSet<JobFlag> jobFlags;
-    private final UUID jobId;
+    private final JobId jobId;
     private final String appName;
 
-    public JobCreateEvent(ClusterControllerService ccs, UUID jobId, String appName, byte[] jobSpec,
+    public JobCreateEvent(ClusterControllerService ccs, JobId jobId, String appName, byte[] jobSpec,
             EnumSet<JobFlag> jobFlags) {
         this.jobId = jobId;
         this.ccs = ccs;
@@ -59,53 +54,29 @@
         if (appCtx == null) {
             throw new HyracksException("No application with id " + appName + " found");
         }
-        JobSpecification spec = appCtx.createJobSpecification(jobId, jobSpec);
-        JobRun run = plan(jobId, spec, appCtx, jobFlags);
-        run.setStatus(JobStatus.INITIALIZED);
+        JobSpecification spec = appCtx.createJobSpecification(jobSpec);
+
+        final JobActivityGraphBuilder builder = new JobActivityGraphBuilder();
+        builder.init(appName, spec, jobFlags);
+        PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
+            @Override
+            public void visit(IOperatorDescriptor op) {
+                op.contributeActivities(builder);
+            }
+        });
+        final JobActivityGraph jag = builder.getActivityGraph();
+
+        JobRun run = new JobRun(jobId, jag);
+
+        run.setStatus(JobStatus.INITIALIZED, null);
 
         ccs.getRunMap().put(jobId, run);
+        JobScheduler jrs = new JobScheduler(ccs, run);
+        run.setScheduler(jrs);
         appCtx.notifyJobCreation(jobId, spec);
     }
 
-    public UUID getJobId() {
+    public JobId getJobId() {
         return jobId;
     }
-
-    private JobRun plan(UUID jobId, JobSpecification jobSpec, final CCApplicationContext appCtx,
-            EnumSet<JobFlag> jobFlags) throws Exception {
-        final JobPlanBuilder builder = new JobPlanBuilder();
-        builder.init(appName, jobId, jobSpec, jobFlags);
-        PlanUtils.visit(jobSpec, new IOperatorDescriptorVisitor() {
-            @Override
-            public void visit(IOperatorDescriptor op) {
-                op.contributeTaskGraph(builder);
-            }
-        });
-        final JobPlan plan = builder.getPlan();
-
-        final Set<ConstraintExpression> contributedConstraints = new HashSet<ConstraintExpression>();
-        final IConstraintExpressionAcceptor acceptor = new IConstraintExpressionAcceptor() {
-            @Override
-            public void addConstraintExpression(ConstraintExpression constraintExpression) {
-                contributedConstraints.add(constraintExpression);
-            }
-        };
-        PlanUtils.visit(jobSpec, new IOperatorDescriptorVisitor() {
-            @Override
-            public void visit(IOperatorDescriptor op) {
-                op.contributeSchedulingConstraints(acceptor, plan, appCtx);
-            }
-        });
-        PlanUtils.visit(jobSpec, new IConnectorDescriptorVisitor() {
-            @Override
-            public void visit(IConnectorDescriptor conn) {
-                conn.contributeSchedulingConstraints(acceptor, plan, appCtx);
-            }
-        });
-        contributedConstraints.addAll(jobSpec.getUserConstraints());
-
-        JobRun run = new JobRun(plan, contributedConstraints);
-
-        return run;
-    }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.java
index 742885e..8f6357c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.java
@@ -14,18 +14,17 @@
  */
 package edu.uci.ics.hyracks.control.cc.job.manager.events;
 
-import java.util.UUID;
-
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 
-public class JobStartEvent extends SynchronizableRunnable {
+public class JobStartEvent extends SynchronizableEvent {
     private final ClusterControllerService ccs;
-    private final UUID jobId;
+    private final JobId jobId;
 
-    public JobStartEvent(ClusterControllerService ccs, UUID jobId) {
+    public JobStartEvent(ClusterControllerService ccs, JobId jobId) {
         this.ccs = ccs;
         this.jobId = jobId;
     }
@@ -39,8 +38,11 @@
         if (run.getStatus() != JobStatus.INITIALIZED) {
             throw new Exception("Job already started");
         }
-        run.setStatus(JobStatus.RUNNING);
-
-        new JobAttemptStartEvent(ccs, jobId).run();
+        run.setStatus(JobStatus.RUNNING, null);
+        try {
+            run.getScheduler().startJob();
+        } catch (Exception e) {
+            ccs.getJobQueue().schedule(new JobCleanupEvent(ccs, run.getJobId(), JobStatus.FAILURE, e));
+        }
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.java
index 348d703..8643e6a 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.java
@@ -15,12 +15,13 @@
 package edu.uci.ics.hyracks.control.cc.job.manager.events;
 
 import java.util.Map;
+import java.util.logging.Level;
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 
-public class NodeHeartbeatEvent extends SynchronizableRunnable {
+public class NodeHeartbeatEvent extends SynchronizableEvent {
     private final ClusterControllerService ccs;
     private final String nodeId;
 
@@ -37,4 +38,9 @@
             state.notifyHeartbeat();
         }
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.FINEST;
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterNodeEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterNodeEvent.java
index 3df302f..5f9377b 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterNodeEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterNodeEvent.java
@@ -20,9 +20,9 @@
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 
-public class RegisterNodeEvent extends SynchronizableRunnable {
+public class RegisterNodeEvent extends SynchronizableEvent {
     private final ClusterControllerService ccs;
     private final String nodeId;
     private final NodeControllerState state;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionAvailibilityEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionAvailibilityEvent.java
new file mode 100644
index 0000000..3f49b97
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionAvailibilityEvent.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.control.cc.job.manager.events;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
+import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
+import edu.uci.ics.hyracks.control.cc.partitions.PartitionUtils;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+
+public class RegisterPartitionAvailibilityEvent extends AbstractEvent {
+    private final ClusterControllerService ccs;
+    private final PartitionDescriptor partitionDescriptor;
+
+    public RegisterPartitionAvailibilityEvent(ClusterControllerService ccs, PartitionDescriptor partitionDescriptor) {
+        this.ccs = ccs;
+        this.partitionDescriptor = partitionDescriptor;
+    }
+
+    @Override
+    public void run() {
+        final PartitionId pid = partitionDescriptor.getPartitionId();
+        JobRun run = ccs.getRunMap().get(pid.getJobId());
+        if (run == null) {
+            return;
+        }
+        PartitionMatchMaker pmm = run.getPartitionMatchMaker();
+        List<Pair<PartitionDescriptor, PartitionRequest>> matches = pmm
+                .registerPartitionDescriptor(partitionDescriptor);
+        for (Pair<PartitionDescriptor, PartitionRequest> match : matches) {
+            PartitionUtils.reportPartitionMatch(ccs, pid, match);
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "PartitionAvailable@" + partitionDescriptor;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionRequestEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionRequestEvent.java
new file mode 100644
index 0000000..bac8dd1
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionRequestEvent.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.control.cc.job.manager.events;
+
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
+import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
+import edu.uci.ics.hyracks.control.cc.partitions.PartitionUtils;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+
+public class RegisterPartitionRequestEvent extends AbstractEvent {
+    private final ClusterControllerService ccs;
+    private final PartitionRequest partitionRequest;
+
+    public RegisterPartitionRequestEvent(ClusterControllerService ccs, PartitionRequest partitionRequest) {
+        this.ccs = ccs;
+        this.partitionRequest = partitionRequest;
+    }
+
+    @Override
+    public void run() {
+        PartitionId pid = partitionRequest.getPartitionId();
+        JobRun run = ccs.getRunMap().get(pid.getJobId());
+        if (run == null) {
+            return;
+        }
+        PartitionMatchMaker pmm = run.getPartitionMatchMaker();
+        Pair<PartitionDescriptor, PartitionRequest> match = pmm.matchPartitionRequest(partitionRequest);
+        if (match != null) {
+            PartitionUtils.reportPartitionMatch(ccs, pid, match);
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "PartitionRequest@" + partitionRequest;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java
index dca0dee..b812594 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java
@@ -17,14 +17,16 @@
 import java.util.HashSet;
 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.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
 
-public class RemoveDeadNodesEvent implements Runnable {
+public class RemoveDeadNodesEvent extends AbstractEvent {
     private static Logger LOGGER = Logger.getLogger(RemoveDeadNodesEvent.class.getName());
 
     private final ClusterControllerService ccs;
@@ -44,15 +46,14 @@
                 LOGGER.info(e.getKey() + " considered dead");
             }
         }
+        Set<JobId> affectedJobIds = new HashSet<JobId>();
         Map<String, Set<String>> ipAddressNodeNameMap = ccs.getIPAddressNodeNameMap();
         for (String deadNode : deadNodes) {
             NodeControllerState state = nodeMap.remove(deadNode);
-            for (final UUID jid : state.getActiveJobIds()) {
-                JobRun run = ccs.getRunMap().get(jid);
-                int lastAttempt = run.getAttempts().size() - 1;
-                LOGGER.info("Aborting: " + jid);
-                ccs.getJobQueue().schedule(new JobAbortEvent(ccs, jid, lastAttempt));
-            }
+
+            // Deal with dead tasks.
+            affectedJobIds.addAll(state.getActiveJobIds());
+
             String ipAddress = state.getNCConfig().dataIPAddress;
             Set<String> ipNodes = ipAddressNodeNameMap.get(ipAddress);
             if (ipNodes != null) {
@@ -61,5 +62,19 @@
                 }
             }
         }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Number of affected jobs: " + affectedJobIds.size());
+        }
+        for (JobId jobId : affectedJobIds) {
+            JobRun run = ccs.getRunMap().get(jobId);
+            if (run != null) {
+                run.getScheduler().notifyNodeFailures(deadNodes);
+            }
+        }
+    }
+
+    @Override
+    public Level logLevel() {
+        return Level.FINE;
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ReportProfilesEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ReportProfilesEvent.java
index 21c16c8..bd82f75 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ReportProfilesEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ReportProfilesEvent.java
@@ -16,14 +16,15 @@
 
 import java.util.List;
 import java.util.Map;
-import java.util.UUID;
+import java.util.logging.Level;
 
-import edu.uci.ics.hyracks.api.job.profiling.om.JobProfile;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
 
-public class ReportProfilesEvent implements Runnable {
+public class ReportProfilesEvent extends AbstractEvent {
     private final ClusterControllerService ccs;
     private final List<JobProfile> profiles;
 
@@ -34,13 +35,14 @@
 
     @Override
     public void run() {
-        Map<UUID, JobRun> runMap = ccs.getRunMap();
+        Map<JobId, JobRun> runMap = ccs.getRunMap();
         for (JobProfile profile : profiles) {
             JobRun run = runMap.get(profile.getJobId());
-            if (run != null) {
-                JobAttempt ja = run.getAttempts().get(profile.getAttempt());
-                ja.getJobProfile().merge(profile);
-            }
         }
     }
+
+    @Override
+    public Level logLevel() {
+        return Level.FINEST;
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ScheduleRunnableStagesEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ScheduleRunnableStagesEvent.java
deleted file mode 100644
index ba1c711..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ScheduleRunnableStagesEvent.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.job.JobStage;
-import edu.uci.ics.hyracks.control.cc.job.JobStageAttempt;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
-import edu.uci.ics.hyracks.control.cc.remote.ops.Phase1Installer;
-import edu.uci.ics.hyracks.control.cc.remote.ops.Phase2Installer;
-import edu.uci.ics.hyracks.control.cc.remote.ops.Phase3Installer;
-import edu.uci.ics.hyracks.control.cc.remote.ops.PortMapMergingAccumulator;
-import edu.uci.ics.hyracks.control.cc.remote.ops.StageStarter;
-import edu.uci.ics.hyracks.control.cc.scheduler.ISchedule;
-
-public class ScheduleRunnableStagesEvent implements Runnable {
-    private static final Logger LOGGER = Logger.getLogger(ScheduleRunnableStagesEvent.class.getName());
-
-    private ClusterControllerService ccs;
-    private UUID jobId;
-    private int attempt;
-
-    public ScheduleRunnableStagesEvent(ClusterControllerService ccs, UUID jobId, int attempt) {
-        this.ccs = ccs;
-        this.jobId = jobId;
-        this.attempt = attempt;
-    }
-
-    @Override
-    public void run() {
-        JobRun run = ccs.getRunMap().get(jobId);
-        JobAttempt ja = run.getAttempts().get(attempt);
-        Set<UUID> pendingStages = ja.getPendingStageIds();
-        Set<UUID> scheduledStages = ja.getInProgressStageIds();
-
-        LOGGER.info(jobId + ":" + attempt + ":Pending stages: " + pendingStages + " Scheduled stages: "
-                + scheduledStages);
-        if (pendingStages.size() == 1 && scheduledStages.isEmpty()) {
-            LOGGER.info(jobId + ":" + attempt + ":No more runnable stages");
-            ccs.getJobQueue().schedule(new JobCleanupEvent(ccs, jobId, attempt, JobStatus.TERMINATED));
-            return;
-        }
-
-        Map<UUID, JobStageAttempt> stageAttemptMap = ja.getStageAttemptMap();
-
-        Set<JobStage> runnableStages = new HashSet<JobStage>();
-        ja.findRunnableStages(runnableStages);
-        LOGGER.info(jobId + ":" + attempt + ": Found " + runnableStages.size() + " runnable stages");
-
-        Set<JobStageAttempt> runnableStageAttempts = new HashSet<JobStageAttempt>();
-        for (JobStage rs : runnableStages) {
-            UUID stageId = rs.getId();
-            LOGGER.info("Runnable Stage: " + jobId + ":" + rs.getId());
-            pendingStages.remove(stageId);
-            scheduledStages.add(stageId);
-            JobStageAttempt jsa = new JobStageAttempt(rs, ja);
-            stageAttemptMap.put(stageId, jsa);
-            runnableStageAttempts.add(jsa);
-        }
-
-        try {
-            ccs.getScheduler().schedule(runnableStageAttempts);
-        } catch (HyracksException e) {
-            e.printStackTrace();
-            ccs.getJobQueue().schedule(new JobAbortEvent(ccs, jobId, attempt));
-            return;
-        }
-
-        final JobPlan plan = run.getJobPlan();
-        for (final JobStageAttempt jsa : runnableStageAttempts) {
-            ISchedule schedule = jsa.getSchedule();
-            final Map<OperatorDescriptorId, Integer> partCountMap = new HashMap<OperatorDescriptorId, Integer>();
-            final Map<String, Map<ActivityNodeId, Set<Integer>>> targetMap = new HashMap<String, Map<ActivityNodeId, Set<Integer>>>();
-            for (ActivityNodeId aid : jsa.getJobStage().getTasks()) {
-                String[] locations = schedule.getPartitions(aid);
-                partCountMap.put(aid.getOperatorDescriptorId(), locations.length);
-                int nLoc = locations.length;
-                for (int i = 0; i < nLoc; ++i) {
-                    Map<ActivityNodeId, Set<Integer>> target = targetMap.get(locations[i]);
-                    if (target == null) {
-                        target = new HashMap<ActivityNodeId, Set<Integer>>();
-                        targetMap.put(locations[i], target);
-                    }
-                    Set<Integer> partIdxs = target.get(aid);
-                    if (partIdxs == null) {
-                        partIdxs = new HashSet<Integer>();
-                        target.put(aid, partIdxs);
-                    }
-                    partIdxs.add(i);
-                }
-            }
-
-            Set<String> participatingNodeIds = ja.getParticipatingNodeIds();
-            for (String nid : targetMap.keySet()) {
-                ccs.getNodeMap().get(nid).getActiveJobIds().add(jobId);
-                participatingNodeIds.add(nid);
-            }
-
-            ccs.getExecutor().execute(new Runnable() {
-                @Override
-                public void run() {
-                    Phase1Installer p1is[] = new Phase1Installer[targetMap.size()];
-                    int i = 0;
-                    for (String nid : targetMap.keySet()) {
-                        p1is[i] = new Phase1Installer(nid, plan.getJobId(), plan.getApplicationName(), plan, jsa
-                                .getJobStage().getId(), jsa.getJobAttempt().getAttempt(), targetMap.get(nid),
-                                partCountMap);
-                        ++i;
-                    }
-                    LOGGER.info("Stage start - Phase 1");
-                    try {
-                        Map<PortInstanceId, Endpoint> globalPortMap = RemoteRunner.runRemote(ccs, p1is,
-                                new PortMapMergingAccumulator());
-
-                        Phase2Installer[] p2is = new Phase2Installer[targetMap.size()];
-                        Phase3Installer[] p3is = new Phase3Installer[targetMap.size()];
-                        StageStarter[] ss = new StageStarter[targetMap.size()];
-
-                        i = 0;
-                        for (String nid : targetMap.keySet()) {
-                            p2is[i] = new Phase2Installer(nid, plan.getJobId(), plan.getApplicationName(), plan, jsa
-                                    .getJobStage().getId(), targetMap.get(nid), partCountMap, globalPortMap);
-                            p3is[i] = new Phase3Installer(nid, plan.getJobId(), jsa.getJobStage().getId());
-                            ss[i] = new StageStarter(nid, plan.getJobId(), jsa.getJobStage().getId());
-                            ++i;
-                        }
-                        LOGGER.info("Stage start - Phase 2");
-                        RemoteRunner.runRemote(ccs, p2is, null);
-                        LOGGER.info("Stage start - Phase 3");
-                        RemoteRunner.runRemote(ccs, p3is, null);
-                        LOGGER.info("Stage start");
-                        RemoteRunner.runRemote(ccs, ss, null);
-                        LOGGER.info("Stage started");
-                    } catch (Exception e) {
-                        e.printStackTrace();
-                    }
-                }
-            });
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletCompleteEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletCompleteEvent.java
deleted file mode 100644
index 8f07d09..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletCompleteEvent.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
-
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.job.JobStageAttempt;
-
-public class StageletCompleteEvent implements Runnable {
-    private final ClusterControllerService ccs;
-    private final UUID jobId;
-    private final UUID stageId;
-    private final int attempt;
-    private final String nodeId;
-
-    public StageletCompleteEvent(ClusterControllerService ccs, UUID jobId, UUID stageId, int attempt, String nodeId) {
-        this.ccs = ccs;
-        this.jobId = jobId;
-        this.stageId = stageId;
-        this.attempt = attempt;
-        this.nodeId = nodeId;
-    }
-
-    @Override
-    public void run() {
-        JobRun run = ccs.getRunMap().get(jobId);
-        JobAttempt jobAttempt = run.getAttempts().get(attempt);
-        JobStageAttempt jsAttempt = jobAttempt.getStageAttemptMap().get(stageId);
-
-        Set<String> participatingNodes = jsAttempt.getParticipatingNodes();
-        Set<String> completedNodes = jsAttempt.getCompletedNodes();
-        completedNodes.add(nodeId);
-
-        if (completedNodes.containsAll(participatingNodes)) {
-            Set<UUID> completedStageIds = jobAttempt.getCompletedStageIds();
-            completedStageIds.add(stageId);
-
-            Set<UUID> inProgressStageIds = jobAttempt.getInProgressStageIds();
-            inProgressStageIds.remove(stageId);
-
-            ccs.getJobQueue().schedule(new ScheduleRunnableStagesEvent(ccs, jobId, attempt));
-        }
-    }
-
-    @Override
-    public String toString() {
-        return "StageletCompleteEvent[" + jobId + ":" + stageId + ":" + attempt + ":" + nodeId + "]";
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletFailureEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletFailureEvent.java
deleted file mode 100644
index 34dbeb4..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletFailureEvent.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
-
-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 edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.job.manager.JobLifecycleHelper;
-
-public class StageletFailureEvent implements Runnable {
-    private final ClusterControllerService ccs;
-    private final UUID jobId;
-    private final UUID stageId;
-    private final int attempt;
-    private final String nodeId;
-
-    public StageletFailureEvent(ClusterControllerService ccs, UUID jobId, UUID stageId, int attempt, String nodeId) {
-        this.ccs = ccs;
-        this.jobId = jobId;
-        this.stageId = stageId;
-        this.attempt = attempt;
-        this.nodeId = nodeId;
-    }
-
-    @Override
-    public void run() {
-        final JobRun run = ccs.getRunMap().get(jobId);
-        List<JobAttempt> attempts = run.getAttempts();
-        JobAttempt ja = attempts.get(attempt);
-        final Set<String> targetNodes = new HashSet<String>(ja.getParticipatingNodeIds());
-        Map<String, NodeControllerState> nodeMap = new HashMap<String, NodeControllerState>();
-        for (String nodeId : targetNodes) {
-            nodeMap.get(nodeId).getActiveJobIds().remove(jobId);
-        }
-        ccs.getExecutor().execute(new Runnable() {
-            @Override
-            public void run() {
-                JobLifecycleHelper.abortJob(ccs, jobId, attempt, targetNodes);
-            }
-        });
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskCompleteEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskCompleteEvent.java
new file mode 100644
index 0000000..19fd95b
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskCompleteEvent.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.control.cc.job.manager.events;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
+
+public class TaskCompleteEvent extends AbstractTaskLifecycleEvent {
+    public TaskCompleteEvent(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId) {
+        super(ccs, jobId, taId, nodeId);
+    }
+
+    @Override
+    protected void performEvent(TaskAttempt ta) {
+        try {
+            ActivityCluster ac = ta.getTaskState().getTaskCluster().getActivityCluster();
+            ac.getJobRun().getScheduler().notifyTaskComplete(ta);
+        } catch (HyracksException e) {
+            e.printStackTrace();
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "TaskCompleteEvent@[" + nodeId + "[" + jobId + ":" + taId + "]";
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskFailureEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskFailureEvent.java
new file mode 100644
index 0000000..8b7e821
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskFailureEvent.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.control.cc.job.manager.events;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
+
+public class TaskFailureEvent extends AbstractTaskLifecycleEvent {
+    private final Exception exception;
+
+    public TaskFailureEvent(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId,
+            Exception exception) {
+        super(ccs, jobId, taId, nodeId);
+        this.exception = exception;
+    }
+
+    @Override
+    protected void performEvent(TaskAttempt ta) {
+        ActivityCluster ac = ta.getTaskState().getTaskCluster().getActivityCluster();
+        ac.getJobRun().getScheduler().notifyTaskFailure(ta, ac, exception);
+    }
+
+    @Override
+    public String toString() {
+        return "TaskFailureEvent[" + jobId + ":" + taId + ":" + nodeId + "]";
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.java
index fc1ba03..b89de07 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.java
@@ -18,9 +18,9 @@
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 
-public class UnregisterNodeEvent extends SynchronizableRunnable {
+public class UnregisterNodeEvent extends SynchronizableEvent {
     private final ClusterControllerService ccs;
     private final String nodeId;
 
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/AbstractEvent.java
similarity index 75%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/AbstractEvent.java
index c884998..5fcd56a 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/AbstractEvent.java
@@ -12,9 +12,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.control.cc.jobqueue;
 
-public enum Direction {
-    INPUT,
-    OUTPUT,
+import java.util.logging.Level;
+
+public abstract class AbstractEvent implements Runnable {
+    public Level logLevel() {
+        return Level.INFO;
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/JobQueue.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/JobQueue.java
index f533ad0..84a844c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/JobQueue.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/JobQueue.java
@@ -20,21 +20,23 @@
 public class JobQueue {
     private static final Logger LOGGER = Logger.getLogger(JobQueue.class.getName());
 
-    private final LinkedBlockingQueue<Runnable> queue;
+    private final LinkedBlockingQueue<AbstractEvent> queue;
     private final JobThread thread;
 
     public JobQueue() {
-        queue = new LinkedBlockingQueue<Runnable>();
+        queue = new LinkedBlockingQueue<AbstractEvent>();
         thread = new JobThread();
         thread.start();
     }
 
-    public void schedule(Runnable runnable) {
-        LOGGER.info("Scheduling: " + runnable);
-        queue.offer(runnable);
+    public void schedule(AbstractEvent event) {
+        if (LOGGER.isLoggable(event.logLevel())) {
+            LOGGER.info("Scheduling: " + event);
+        }
+        queue.offer(event);
     }
 
-    public void scheduleAndSync(SynchronizableRunnable sRunnable) throws Exception {
+    public void scheduleAndSync(SynchronizableEvent sRunnable) throws Exception {
         schedule(sRunnable);
         sRunnable.sync();
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableEvent.java
new file mode 100644
index 0000000..88c9097
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableEvent.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.jobqueue;
+
+public abstract class SynchronizableEvent extends AbstractEvent {
+    private boolean done;
+
+    private Exception e;
+
+    protected abstract void doRun() throws Exception;
+
+    public void init() {
+        done = false;
+        e = null;
+    }
+
+    @Override
+    public final void run() {
+        try {
+            doRun();
+        } catch (Exception e) {
+            this.e = e;
+        } finally {
+            synchronized (this) {
+                done = true;
+                notifyAll();
+            }
+        }
+    }
+
+    public final synchronized void sync() throws Exception {
+        while (!done) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+            }
+        }
+        if (e != null) {
+            throw e;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableRunnable.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableRunnable.java
deleted file mode 100644
index 53315b7..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableRunnable.java
+++ /dev/null
@@ -1,41 +0,0 @@
-package edu.uci.ics.hyracks.control.cc.jobqueue;
-
-public abstract class SynchronizableRunnable implements Runnable {
-    private boolean done;
-
-    private Exception e;
-
-    protected abstract void doRun() throws Exception;
-
-    public void init() {
-        done = false;
-        e = null;
-    }
-
-    @Override
-    public final void run() {
-        try {
-            doRun();
-        } catch (Exception e) {
-            this.e = e;
-        } finally {
-            synchronized (this) {
-                done = true;
-                notifyAll();
-            }
-        }
-    }
-
-    public final synchronized void sync() throws Exception {
-        while (!done) {
-            try {
-                wait();
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-            }
-        }
-        if (e != null) {
-            throw e;
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java
new file mode 100644
index 0000000..d2c14b2
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java
@@ -0,0 +1,201 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.partitions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
+
+public class PartitionMatchMaker {
+    private final Map<PartitionId, List<PartitionDescriptor>> partitionDescriptors;
+
+    private final Map<PartitionId, List<PartitionRequest>> partitionRequests;
+
+    public PartitionMatchMaker() {
+        partitionDescriptors = new HashMap<PartitionId, List<PartitionDescriptor>>();
+        partitionRequests = new HashMap<PartitionId, List<PartitionRequest>>();
+    }
+
+    public List<Pair<PartitionDescriptor, PartitionRequest>> registerPartitionDescriptor(
+            PartitionDescriptor partitionDescriptor) {
+        List<Pair<PartitionDescriptor, PartitionRequest>> matches = new ArrayList<Pair<PartitionDescriptor, PartitionRequest>>();
+        PartitionId pid = partitionDescriptor.getPartitionId();
+        boolean matched = false;
+        List<PartitionRequest> requests = partitionRequests.get(pid);
+        if (requests != null) {
+            Iterator<PartitionRequest> i = requests.iterator();
+            while (i.hasNext()) {
+                PartitionRequest req = i.next();
+                if (partitionDescriptor.getState().isAtLeast(req.getMinimumState())) {
+                    matches.add(new Pair<PartitionDescriptor, PartitionRequest>(partitionDescriptor, req));
+                    i.remove();
+                    matched = true;
+                    if (!partitionDescriptor.isReusable()) {
+                        break;
+                    }
+                }
+            }
+            if (requests.isEmpty()) {
+                partitionRequests.remove(pid);
+            }
+        }
+
+        if (!matched) {
+            List<PartitionDescriptor> descriptors = partitionDescriptors.get(pid);
+            if (descriptors == null) {
+                descriptors = new ArrayList<PartitionDescriptor>();
+                partitionDescriptors.put(pid, descriptors);
+            }
+            descriptors.add(partitionDescriptor);
+        }
+
+        return matches;
+    }
+
+    public Pair<PartitionDescriptor, PartitionRequest> matchPartitionRequest(PartitionRequest partitionRequest) {
+        Pair<PartitionDescriptor, PartitionRequest> match = null;
+
+        PartitionId pid = partitionRequest.getPartitionId();
+
+        List<PartitionDescriptor> descriptors = partitionDescriptors.get(pid);
+        if (descriptors != null) {
+            Iterator<PartitionDescriptor> i = descriptors.iterator();
+            while (i.hasNext()) {
+                PartitionDescriptor descriptor = i.next();
+                if (descriptor.getState().isAtLeast(partitionRequest.getMinimumState())) {
+                    match = new Pair<PartitionDescriptor, PartitionRequest>(descriptor, partitionRequest);
+                    if (!descriptor.isReusable()) {
+                        i.remove();
+                    }
+                    break;
+                }
+            }
+            if (descriptors.isEmpty()) {
+                partitionDescriptors.remove(pid);
+            }
+        }
+
+        if (match == null) {
+            List<PartitionRequest> requests = partitionRequests.get(pid);
+            if (requests == null) {
+                requests = new ArrayList<PartitionRequest>();
+                partitionRequests.put(pid, requests);
+            }
+            requests.add(partitionRequest);
+        }
+
+        return match;
+    }
+
+    public PartitionState getMaximumAvailableState(PartitionId pid) {
+        List<PartitionDescriptor> descriptors = partitionDescriptors.get(pid);
+        if (descriptors == null) {
+            return null;
+        }
+        for (PartitionDescriptor descriptor : descriptors) {
+            if (descriptor.getState() == PartitionState.COMMITTED) {
+                return PartitionState.COMMITTED;
+            }
+        }
+        return PartitionState.STARTED;
+    }
+
+    private interface IEntryFilter<T> {
+        public boolean matches(T o);
+    }
+
+    private static <T> void removeEntries(List<T> list, IEntryFilter<T> filter) {
+        Iterator<T> j = list.iterator();
+        while (j.hasNext()) {
+            T o = j.next();
+            if (filter.matches(o)) {
+                j.remove();
+            }
+        }
+    }
+
+    private static <T> void removeEntries(Map<PartitionId, List<T>> map, IEntryFilter<T> filter) {
+        Iterator<Map.Entry<PartitionId, List<T>>> i = map.entrySet().iterator();
+        while (i.hasNext()) {
+            Map.Entry<PartitionId, List<T>> e = i.next();
+            List<T> list = e.getValue();
+            removeEntries(list, filter);
+            if (list.isEmpty()) {
+                i.remove();
+            }
+        }
+    }
+
+    public void notifyNodeFailures(final Set<String> deadNodes) {
+        removeEntries(partitionDescriptors, new IEntryFilter<PartitionDescriptor>() {
+            @Override
+            public boolean matches(PartitionDescriptor o) {
+                return deadNodes.contains(o.getNodeId());
+            }
+        });
+        removeEntries(partitionRequests, new IEntryFilter<PartitionRequest>() {
+            @Override
+            public boolean matches(PartitionRequest o) {
+                return deadNodes.contains(o.getNodeId());
+            }
+        });
+    }
+
+    public void removeUncommittedPartitions(Set<PartitionId> partitionIds, final Set<TaskAttemptId> taIds) {
+        IEntryFilter<PartitionDescriptor> filter = new IEntryFilter<PartitionDescriptor>() {
+            @Override
+            public boolean matches(PartitionDescriptor o) {
+                return o.getState() != PartitionState.COMMITTED && taIds.contains(o.getProducingTaskAttemptId());
+            }
+        };
+        for (PartitionId pid : partitionIds) {
+            List<PartitionDescriptor> descriptors = partitionDescriptors.get(pid);
+            if (descriptors != null) {
+                removeEntries(descriptors, filter);
+                if (descriptors.isEmpty()) {
+                    partitionDescriptors.remove(pid);
+                }
+            }
+        }
+    }
+
+    public void removePartitionRequests(Set<PartitionId> partitionIds, final Set<TaskAttemptId> taIds) {
+        IEntryFilter<PartitionRequest> filter = new IEntryFilter<PartitionRequest>() {
+            @Override
+            public boolean matches(PartitionRequest o) {
+                return taIds.contains(o.getRequestingTaskAttemptId());
+            }
+        };
+        for (PartitionId pid : partitionIds) {
+            List<PartitionRequest> requests = partitionRequests.get(pid);
+            if (requests != null) {
+                removeEntries(requests, filter);
+                if (requests.isEmpty()) {
+                    partitionRequests.remove(pid);
+                }
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java
new file mode 100644
index 0000000..90c6ad1
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java
@@ -0,0 +1,33 @@
+package edu.uci.ics.hyracks.control.cc.partitions;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+
+public class PartitionUtils {
+    public static void reportPartitionMatch(ClusterControllerService ccs, final PartitionId pid,
+            Pair<PartitionDescriptor, PartitionRequest> match) {
+        PartitionDescriptor desc = match.first;
+        PartitionRequest req = match.second;
+
+        NodeControllerState producerNCS = ccs.getNodeMap().get(desc.getNodeId());
+        NodeControllerState requestorNCS = ccs.getNodeMap().get(req.getNodeId());
+        final NetworkAddress dataport = producerNCS.getDataPort();
+        final INodeController requestorNC = requestorNCS.getNodeController();
+        ccs.getExecutor().execute(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    requestorNC.reportPartitionAvailability(pid, dataport);
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        });
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
index c0a35cb..ec15186 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.cc.remote;
 
-import edu.uci.ics.hyracks.api.control.INodeController;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
 
 public interface RemoteOp<T> {
     public String getNodeId();
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java
index 9bd4375..a37c786 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java
@@ -18,9 +18,9 @@
 import java.util.Vector;
 import java.util.concurrent.Semaphore;
 
-import edu.uci.ics.hyracks.api.control.INodeController;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
 
 public class RemoteRunner {
     public static <T, R> R runRemote(ClusterControllerService ccs, final RemoteOp<T>[] remoteOps,
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java
index f8ca6af..52e726c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java
@@ -14,8 +14,8 @@
  */
 package edu.uci.ics.hyracks.control.cc.remote.ops;
 
-import edu.uci.ics.hyracks.api.control.INodeController;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
 
 public class ApplicationDestroyer implements RemoteOp<Void> {
     private String nodeId;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java
index af61249..d6a1d26 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java
@@ -14,8 +14,8 @@
  */
 package edu.uci.ics.hyracks.control.cc.remote.ops;
 
-import edu.uci.ics.hyracks.api.control.INodeController;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
 
 public class ApplicationStarter implements RemoteOp<Void> {
     private String nodeId;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java
index 81a8610..414cf12 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java
@@ -14,16 +14,15 @@
  */
 package edu.uci.ics.hyracks.control.cc.remote.ops;
 
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.control.INodeController;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
 
 public class JobCompleteNotifier implements RemoteOp<Void> {
     private String nodeId;
-    private UUID jobId;
+    private JobId jobId;
 
-    public JobCompleteNotifier(String nodeId, UUID jobId) {
+    public JobCompleteNotifier(String nodeId, JobId jobId) {
         this.nodeId = nodeId;
         this.jobId = jobId;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobletAborter.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobletAborter.java
deleted file mode 100644
index b50aa8e..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobletAborter.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.remote.ops;
-
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-
-public class JobletAborter implements RemoteOp<Void> {
-    private String nodeId;
-    private UUID jobId;
-    private int attempt;
-
-    public JobletAborter(String nodeId, UUID jobId, int attempt) {
-        this.nodeId = nodeId;
-        this.jobId = jobId;
-        this.attempt = attempt;
-    }
-
-    @Override
-    public Void execute(INodeController node) throws Exception {
-        node.abortJoblet(jobId, attempt);
-        return null;
-    }
-
-    @Override
-    public String toString() {
-        return jobId + " Aborting";
-    }
-
-    @Override
-    public String getNodeId() {
-        return nodeId;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase1Installer.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase1Installer.java
deleted file mode 100644
index 2fe67ac..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase1Installer.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.remote.ops;
-
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-
-public class Phase1Installer implements RemoteOp<Map<PortInstanceId, Endpoint>> {
-    private String nodeId;
-    private UUID jobId;
-    private String appName;
-    private JobPlan plan;
-    private UUID stageId;
-    private int attempt;
-    private Map<ActivityNodeId, Set<Integer>> tasks;
-    private Map<OperatorDescriptorId, Integer> opNumPartitions;
-
-    public Phase1Installer(String nodeId, UUID jobId, String appName, JobPlan plan, UUID stageId, int attempt,
-            Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Integer> opNumPartitions) {
-        this.nodeId = nodeId;
-        this.jobId = jobId;
-        this.appName = appName;
-        this.plan = plan;
-        this.stageId = stageId;
-        this.attempt = attempt;
-        this.tasks = tasks;
-        this.opNumPartitions = opNumPartitions;
-    }
-
-    @Override
-    public Map<PortInstanceId, Endpoint> execute(INodeController node) throws Exception {
-        return node.initializeJobletPhase1(appName, jobId, attempt, JavaSerializationUtils.serialize(plan), stageId,
-                tasks, opNumPartitions);
-    }
-
-    @Override
-    public String toString() {
-        return jobId + " Distribution Phase 1";
-    }
-
-    @Override
-    public String getNodeId() {
-        return nodeId;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase2Installer.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase2Installer.java
deleted file mode 100644
index 99d7c9f..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase2Installer.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.remote.ops;
-
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-
-public class Phase2Installer implements RemoteOp<Void> {
-    private String nodeId;
-    private UUID jobId;
-    private String appName;
-    private JobPlan plan;
-    private UUID stageId;
-    private Map<ActivityNodeId, Set<Integer>> tasks;
-    private Map<OperatorDescriptorId, Integer> opNumPartitions;
-    private Map<PortInstanceId, Endpoint> globalPortMap;
-
-    public Phase2Installer(String nodeId, UUID jobId, String appName, JobPlan plan, UUID stageId,
-            Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Integer> opNumPartitions,
-            Map<PortInstanceId, Endpoint> globalPortMap) {
-        this.nodeId = nodeId;
-        this.jobId = jobId;
-        this.appName = appName;
-        this.plan = plan;
-        this.stageId = stageId;
-        this.tasks = tasks;
-        this.opNumPartitions = opNumPartitions;
-        this.globalPortMap = globalPortMap;
-    }
-
-    @Override
-    public Void execute(INodeController node) throws Exception {
-        node.initializeJobletPhase2(appName, jobId, JavaSerializationUtils.serialize(plan), stageId, tasks,
-                opNumPartitions, globalPortMap);
-        return null;
-    }
-
-    @Override
-    public String toString() {
-        return jobId + " Distribution Phase 2";
-    }
-
-    @Override
-    public String getNodeId() {
-        return nodeId;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase3Installer.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase3Installer.java
deleted file mode 100644
index 02f9aa1..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase3Installer.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.remote.ops;
-
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-
-public class Phase3Installer implements RemoteOp<Void> {
-    private String nodeId;
-    private UUID jobId;
-    private UUID stageId;
-
-    public Phase3Installer(String nodeId, UUID jobId, UUID stageId) {
-        this.nodeId = nodeId;
-        this.jobId = jobId;
-        this.stageId = stageId;
-    }
-
-    @Override
-    public Void execute(INodeController node) throws Exception {
-        node.commitJobletInitialization(jobId, stageId);
-        return null;
-    }
-
-    @Override
-    public String toString() {
-        return jobId + " Distribution Phase 3";
-    }
-
-    @Override
-    public String getNodeId() {
-        return nodeId;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/PortMapMergingAccumulator.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/PortMapMergingAccumulator.java
deleted file mode 100644
index 8cffd08..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/PortMapMergingAccumulator.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.remote.ops;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
-import edu.uci.ics.hyracks.control.cc.remote.Accumulator;
-
-public 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;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/StageStarter.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/StageStarter.java
deleted file mode 100644
index d62254b..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/StageStarter.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.remote.ops;
-
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-
-public class StageStarter implements RemoteOp<Void> {
-    private String nodeId;
-    private UUID jobId;
-    private UUID stageId;
-
-    public StageStarter(String nodeId, UUID jobId, UUID stageId) {
-        this.nodeId = nodeId;
-        this.jobId = jobId;
-        this.stageId = stageId;
-    }
-
-    @Override
-    public Void execute(INodeController node) throws Exception {
-        node.startStage(jobId, stageId);
-        return null;
-    }
-
-    @Override
-    public String toString() {
-        return jobId + " Started Stage: " + stageId;
-    }
-
-    @Override
-    public String getNodeId() {
-        return nodeId;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java
new file mode 100644
index 0000000..0e59c8df
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.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.control.cc.scheduler;
+
+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.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.ActivityClusterId;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+
+public class ActivityClusterGraphBuilder {
+    private static final Logger LOGGER = Logger.getLogger(ActivityClusterGraphBuilder.class.getName());
+
+    private final JobRun jobRun;
+
+    public ActivityClusterGraphBuilder(JobRun jobRun) {
+        this.jobRun = jobRun;
+    }
+
+    private static Pair<ActivityId, ActivityId> findMergePair(JobActivityGraph jag, JobSpecification spec,
+            Set<ActivityCluster> eqSets) {
+        Map<ActivityId, IActivity> activityNodeMap = jag.getActivityNodeMap();
+        for (ActivityCluster eqSet : eqSets) {
+            for (ActivityId t : eqSet.getActivities()) {
+                IActivity activity = activityNodeMap.get(t);
+                List<Integer> inputList = jag.getActivityInputMap().get(t);
+                if (inputList != null) {
+                    for (Integer idx : inputList) {
+                        IConnectorDescriptor conn = spec.getInputConnectorDescriptor(activity.getActivityId()
+                                .getOperatorDescriptorId(), idx);
+                        OperatorDescriptorId producerId = spec.getProducer(conn).getOperatorId();
+                        int producerOutputIndex = spec.getProducerOutputIndex(conn);
+                        ActivityId inTask = jag.getOperatorOutputMap().get(producerId).get(producerOutputIndex);
+                        if (!eqSet.getActivities().contains(inTask)) {
+                            return new Pair<ActivityId, ActivityId>(t, inTask);
+                        }
+                    }
+                }
+                List<Integer> outputList = jag.getActivityOutputMap().get(t);
+                if (outputList != null) {
+                    for (Integer idx : outputList) {
+                        IConnectorDescriptor conn = spec.getOutputConnectorDescriptor(activity.getActivityId()
+                                .getOperatorDescriptorId(), idx);
+                        OperatorDescriptorId consumerId = spec.getConsumer(conn).getOperatorId();
+                        int consumerInputIndex = spec.getConsumerInputIndex(conn);
+                        ActivityId outTask = jag.getOperatorInputMap().get(consumerId).get(consumerInputIndex);
+                        if (!eqSet.getActivities().contains(outTask)) {
+                            return new Pair<ActivityId, ActivityId>(t, outTask);
+                        }
+                    }
+                }
+            }
+        }
+        return null;
+    }
+
+    public Set<ActivityCluster> inferActivityClusters(JobActivityGraph jag) {
+        JobSpecification spec = jag.getJobSpecification();
+
+        /*
+         * Build initial equivalence sets map. We create a map such that for each IOperatorTask, t -> { t }
+         */
+        Map<ActivityId, ActivityCluster> stageMap = new HashMap<ActivityId, ActivityCluster>();
+        Set<ActivityCluster> stages = new HashSet<ActivityCluster>();
+        for (Set<ActivityId> taskIds : jag.getOperatorActivityMap().values()) {
+            for (ActivityId taskId : taskIds) {
+                Set<ActivityId> eqSet = new HashSet<ActivityId>();
+                eqSet.add(taskId);
+                ActivityCluster stage = new ActivityCluster(jobRun, eqSet);
+                stageMap.put(taskId, stage);
+                stages.add(stage);
+            }
+        }
+
+        boolean changed = true;
+        while (changed) {
+            changed = false;
+            Pair<ActivityId, ActivityId> pair = findMergePair(jag, spec, stages);
+            if (pair != null) {
+                merge(stageMap, stages, pair.first, pair.second);
+                changed = true;
+            }
+        }
+
+        Map<ActivityId, Set<ActivityId>> blocker2BlockedMap = jag.getBlocker2BlockedMap();
+        for (ActivityCluster s : stages) {
+            Set<ActivityCluster> blockedStages = new HashSet<ActivityCluster>();
+            for (ActivityId t : s.getActivities()) {
+                Set<ActivityId> blockedTasks = blocker2BlockedMap.get(t);
+                if (blockedTasks != null) {
+                    for (ActivityId bt : blockedTasks) {
+                        blockedStages.add(stageMap.get(bt));
+                    }
+                }
+            }
+            for (ActivityCluster bs : blockedStages) {
+                bs.addDependency(s);
+                s.addDependent(bs);
+            }
+        }
+        Set<ActivityCluster> roots = new HashSet<ActivityCluster>();
+        int idCounter = 0;
+        for (ActivityCluster s : stages) {
+            s.setActivityClusterId(new ActivityClusterId(idCounter++));
+            if (s.getDependents().isEmpty()) {
+                roots.add(s);
+            }
+        }
+        jobRun.setActivityClusters(stages);
+        jobRun.getActivityClusterMap().putAll(stageMap);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Inferred " + stages.size() + " stages");
+            for (ActivityCluster s : stages) {
+                LOGGER.info(s.toString());
+            }
+        }
+        return roots;
+    }
+
+    private void merge(Map<ActivityId, ActivityCluster> eqSetMap, Set<ActivityCluster> eqSets, ActivityId t1,
+            ActivityId t2) {
+        ActivityCluster stage1 = eqSetMap.get(t1);
+        Set<ActivityId> s1 = stage1.getActivities();
+        ActivityCluster stage2 = eqSetMap.get(t2);
+        Set<ActivityId> s2 = stage2.getActivities();
+
+        Set<ActivityId> mergedSet = new HashSet<ActivityId>();
+        mergedSet.addAll(s1);
+        mergedSet.addAll(s2);
+
+        eqSets.remove(stage1);
+        eqSets.remove(stage2);
+        ActivityCluster mergedStage = new ActivityCluster(jobRun, mergedSet);
+        eqSets.add(mergedStage);
+
+        for (ActivityId t : mergedSet) {
+            eqSetMap.put(t, mergedStage);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
new file mode 100644
index 0000000..e5d9f64
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
@@ -0,0 +1,337 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.scheduler;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+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.constraints.expressions.LValueConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
+import edu.uci.ics.hyracks.api.dataflow.connectors.PipeliningConnectorPolicy;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.ActivityClusterPlan;
+import edu.uci.ics.hyracks.control.cc.job.ActivityPlan;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.job.Task;
+import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
+import edu.uci.ics.hyracks.control.cc.job.TaskClusterId;
+
+public class ActivityClusterPlanner {
+    private static final Logger LOGGER = Logger.getLogger(ActivityClusterPlanner.class.getName());
+
+    private final JobScheduler scheduler;
+
+    private final Map<PartitionId, TaskCluster> partitionProducingTaskClusterMap;
+
+    public ActivityClusterPlanner(JobScheduler newJobScheduler) {
+        this.scheduler = newJobScheduler;
+        partitionProducingTaskClusterMap = new HashMap<PartitionId, TaskCluster>();
+    }
+
+    public void planActivityCluster(ActivityCluster ac) throws HyracksException {
+        JobRun jobRun = scheduler.getJobRun();
+        Map<ActivityId, ActivityPartitionDetails> pcMap = computePartitionCounts(ac);
+
+        Map<ActivityId, ActivityPlan> activityPlanMap = new HashMap<ActivityId, ActivityPlan>();
+        Set<ActivityId> activities = ac.getActivities();
+
+        Map<TaskId, Set<TaskId>> taskClusterMap = new HashMap<TaskId, Set<TaskId>>();
+
+        Set<ActivityId> depAnIds = new HashSet<ActivityId>();
+        for (ActivityId anId : activities) {
+            depAnIds.clear();
+            getDependencyActivityIds(depAnIds, anId);
+            ActivityPartitionDetails apd = pcMap.get(anId);
+            Task[] tasks = new Task[apd.getPartitionCount()];
+            ActivityPlan activityPlan = new ActivityPlan(apd);
+            for (int i = 0; i < tasks.length; ++i) {
+                TaskId tid = new TaskId(anId, i);
+                tasks[i] = new Task(tid, activityPlan);
+                for (ActivityId danId : depAnIds) {
+                    ActivityCluster dAC = jobRun.getActivityClusterMap().get(danId);
+                    ActivityClusterPlan dACP = dAC.getPlan();
+                    assert dACP != null : "IllegalStateEncountered: Dependent AC is being planned without a plan for dependency AC: Encountered no plan for ActivityID "
+                            + danId;
+                    Task[] dATasks = dACP.getActivityPlanMap().get(danId).getTasks();
+                    assert dATasks != null : "IllegalStateEncountered: Dependent AC is being planned without a plan for dependency AC: Encountered no plan for ActivityID "
+                            + danId;
+                    assert dATasks.length == tasks.length : "Dependency activity partitioned differently from dependent: "
+                            + dATasks.length + " != " + tasks.length;
+                    Task dTask = dATasks[i];
+                    TaskId dTaskId = dTask.getTaskId();
+                    tasks[i].getDependencies().add(dTaskId);
+                    dTask.getDependents().add(tid);
+                }
+                Set<TaskId> cluster = new HashSet<TaskId>();
+                cluster.add(tid);
+                taskClusterMap.put(tid, cluster);
+            }
+            activityPlan.setTasks(tasks);
+            activityPlanMap.put(anId, activityPlan);
+        }
+
+        Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = assignConnectorPolicy(ac, activityPlanMap);
+        scheduler.getJobRun().getConnectorPolicyMap().putAll(connectorPolicies);
+
+        Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> taskConnectivity = new HashMap<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>>();
+        JobActivityGraph jag = jobRun.getJobActivityGraph();
+        BitSet targetBitmap = new BitSet();
+        for (ActivityId ac1 : activities) {
+            Task[] ac1TaskStates = activityPlanMap.get(ac1).getTasks();
+            int nProducers = ac1TaskStates.length;
+            List<IConnectorDescriptor> outputConns = jag.getActivityOutputConnectorDescriptors(ac1);
+            if (outputConns != null) {
+                for (IConnectorDescriptor c : outputConns) {
+                    ConnectorDescriptorId cdId = c.getConnectorId();
+                    ActivityId ac2 = jag.getConsumerActivity(cdId);
+                    Task[] ac2TaskStates = activityPlanMap.get(ac2).getTasks();
+                    int nConsumers = ac2TaskStates.length;
+                    for (int i = 0; i < nProducers; ++i) {
+                        c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
+                        List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = taskConnectivity.get(ac1TaskStates[i]
+                                .getTaskId());
+                        if (cInfoList == null) {
+                            cInfoList = new ArrayList<Pair<TaskId, ConnectorDescriptorId>>();
+                            taskConnectivity.put(ac1TaskStates[i].getTaskId(), cInfoList);
+                        }
+                        Set<TaskId> cluster = taskClusterMap.get(ac1TaskStates[i].getTaskId());
+                        for (int j = targetBitmap.nextSetBit(0); j >= 0; j = targetBitmap.nextSetBit(j + 1)) {
+                            cInfoList.add(new Pair<TaskId, ConnectorDescriptorId>(ac2TaskStates[j].getTaskId(), cdId));
+                            IConnectorPolicy cPolicy = connectorPolicies.get(cdId);
+                            if (cPolicy.requiresProducerConsumerCoscheduling()) {
+                                cluster.add(ac2TaskStates[j].getTaskId());
+                            }
+                        }
+                    }
+                }
+            }
+        }
+
+        boolean done = false;
+        while (!done) {
+            done = true;
+            Set<TaskId> set = new HashSet<TaskId>();
+            Set<TaskId> oldSet = null;
+            for (Map.Entry<TaskId, Set<TaskId>> e : taskClusterMap.entrySet()) {
+                set.clear();
+                oldSet = e.getValue();
+                set.addAll(e.getValue());
+                for (TaskId tid : e.getValue()) {
+                    set.addAll(taskClusterMap.get(tid));
+                }
+                for (TaskId tid : set) {
+                    Set<TaskId> targetSet = taskClusterMap.get(tid);
+                    if (!targetSet.equals(set)) {
+                        done = false;
+                        break;
+                    }
+                }
+                if (!done) {
+                    break;
+                }
+            }
+            for (TaskId tid : oldSet) {
+                taskClusterMap.put(tid, set);
+            }
+        }
+
+        Set<Set<TaskId>> clusters = new HashSet<Set<TaskId>>(taskClusterMap.values());
+        Set<TaskCluster> tcSet = new HashSet<TaskCluster>();
+        int counter = 0;
+        for (Set<TaskId> cluster : clusters) {
+            Set<Task> taskStates = new HashSet<Task>();
+            for (TaskId tid : cluster) {
+                taskStates.add(activityPlanMap.get(tid.getActivityId()).getTasks()[tid.getPartition()]);
+            }
+            TaskCluster tc = new TaskCluster(new TaskClusterId(ac.getActivityClusterId(), counter++), ac,
+                    taskStates.toArray(new Task[taskStates.size()]));
+            tcSet.add(tc);
+            for (TaskId tid : cluster) {
+                activityPlanMap.get(tid.getActivityId()).getTasks()[tid.getPartition()].setTaskCluster(tc);
+            }
+        }
+        TaskCluster[] taskClusters = tcSet.toArray(new TaskCluster[tcSet.size()]);
+
+        for (TaskCluster tc : taskClusters) {
+            Set<TaskCluster> tcDependencyTaskClusters = tc.getDependencyTaskClusters();
+            for (Task ts : tc.getTasks()) {
+                TaskId tid = ts.getTaskId();
+                List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = taskConnectivity.get(tid);
+                if (cInfoList != null) {
+                    for (Pair<TaskId, ConnectorDescriptorId> p : cInfoList) {
+                        Task targetTS = activityPlanMap.get(p.first.getActivityId()).getTasks()[p.first.getPartition()];
+                        TaskCluster targetTC = targetTS.getTaskCluster();
+                        if (targetTC != tc) {
+                            ConnectorDescriptorId cdId = p.second;
+                            PartitionId pid = new PartitionId(jobRun.getJobId(), cdId, tid.getPartition(),
+                                    p.first.getPartition());
+                            tc.getProducedPartitions().add(pid);
+                            targetTC.getRequiredPartitions().add(pid);
+                            partitionProducingTaskClusterMap.put(pid, tc);
+                        }
+                    }
+                }
+
+                for (TaskId dTid : ts.getDependencies()) {
+                    TaskCluster dTC = getTaskCluster(dTid);
+                    dTC.getDependentTaskClusters().add(tc);
+                    tcDependencyTaskClusters.add(dTC);
+                }
+            }
+        }
+
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Plan for " + ac);
+            LOGGER.info("Built " + tcSet.size() + " Task Clusters");
+            for (TaskCluster tc : tcSet) {
+                LOGGER.info("Tasks: " + Arrays.toString(tc.getTasks()));
+            }
+        }
+
+        ac.setPlan(new ActivityClusterPlan(taskClusters, activityPlanMap));
+    }
+
+    private TaskCluster getTaskCluster(TaskId tid) {
+        ActivityCluster ac = scheduler.getJobRun().getActivityClusterMap().get(tid.getActivityId());
+        ActivityClusterPlan acp = ac.getPlan();
+        Task[] tasks = acp.getActivityPlanMap().get(tid.getActivityId()).getTasks();
+        Task task = tasks[tid.getPartition()];
+        assert task.getTaskId().equals(tid);
+        return task.getTaskCluster();
+    }
+
+    private void getDependencyActivityIds(Set<ActivityId> depAnIds, ActivityId anId) {
+        JobActivityGraph jag = scheduler.getJobRun().getJobActivityGraph();
+        Set<ActivityId> blockers = jag.getBlocked2BlockerMap().get(anId);
+        if (blockers != null) {
+            depAnIds.addAll(blockers);
+        }
+    }
+
+    private Map<ConnectorDescriptorId, IConnectorPolicy> assignConnectorPolicy(ActivityCluster ac,
+            Map<ActivityId, ActivityPlan> taskMap) {
+        JobActivityGraph jag = scheduler.getJobRun().getJobActivityGraph();
+        Map<ConnectorDescriptorId, IConnectorPolicy> cPolicyMap = new HashMap<ConnectorDescriptorId, IConnectorPolicy>();
+        Set<ActivityId> activities = ac.getActivities();
+        BitSet targetBitmap = new BitSet();
+        for (ActivityId a1 : activities) {
+            Task[] ac1TaskStates = taskMap.get(a1).getTasks();
+            int nProducers = ac1TaskStates.length;
+            List<IConnectorDescriptor> outputConns = jag.getActivityOutputConnectorDescriptors(a1);
+            if (outputConns != null) {
+                for (IConnectorDescriptor c : outputConns) {
+                    ConnectorDescriptorId cdId = c.getConnectorId();
+                    ActivityId a2 = jag.getConsumerActivity(cdId);
+                    Task[] ac2TaskStates = taskMap.get(a2).getTasks();
+                    int nConsumers = ac2TaskStates.length;
+
+                    int[] fanouts = new int[nProducers];
+                    for (int i = 0; i < nProducers; ++i) {
+                        c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
+                        fanouts[i] = targetBitmap.cardinality();
+                    }
+                    IConnectorPolicy cp = assignConnectorPolicy(c, nProducers, nConsumers, fanouts);
+                    cPolicyMap.put(cdId, cp);
+                }
+            }
+        }
+        return cPolicyMap;
+    }
+
+    private IConnectorPolicy assignConnectorPolicy(IConnectorDescriptor c, int nProducers, int nConsumers, int[] fanouts) {
+        IConnectorPolicyAssignmentPolicy cpap = scheduler.getJobRun().getJobActivityGraph().getJobSpecification()
+                .getConnectorPolicyAssignmentPolicy();
+        if (cpap != null) {
+            return cpap.getConnectorPolicyAssignment(c, nProducers, nConsumers, fanouts);
+        }
+        return new PipeliningConnectorPolicy();
+    }
+
+    private Map<ActivityId, ActivityPartitionDetails> computePartitionCounts(ActivityCluster ac)
+            throws HyracksException {
+        PartitionConstraintSolver solver = scheduler.getSolver();
+        JobRun jobRun = scheduler.getJobRun();
+        Set<LValueConstraintExpression> lValues = new HashSet<LValueConstraintExpression>();
+        for (ActivityId anId : ac.getActivities()) {
+            lValues.add(new PartitionCountExpression(anId.getOperatorDescriptorId()));
+        }
+        solver.solve(lValues);
+        Map<OperatorDescriptorId, Integer> nPartMap = new HashMap<OperatorDescriptorId, Integer>();
+        for (LValueConstraintExpression lv : lValues) {
+            Object value = solver.getValue(lv);
+            if (value == null) {
+                throw new HyracksException("No value found for " + lv);
+            }
+            if (!(value instanceof Number)) {
+                throw new HyracksException("Unexpected type of value bound to " + lv + ": " + value.getClass() + "("
+                        + value + ")");
+            }
+            int nParts = ((Number) value).intValue();
+            if (nParts <= 0) {
+                throw new HyracksException("Unsatisfiable number of partitions for " + lv + ": " + nParts);
+            }
+            nPartMap.put(((PartitionCountExpression) lv).getOperatorDescriptorId(), Integer.valueOf(nParts));
+        }
+        Map<ActivityId, ActivityPartitionDetails> activityPartsMap = new HashMap<ActivityId, ActivityPartitionDetails>();
+        for (ActivityId anId : ac.getActivities()) {
+            int nParts = nPartMap.get(anId.getOperatorDescriptorId());
+            int[] nInputPartitions = null;
+            List<IConnectorDescriptor> inputs = jobRun.getJobActivityGraph().getActivityInputConnectorDescriptors(anId);
+            if (inputs != null) {
+                nInputPartitions = new int[inputs.size()];
+                for (int i = 0; i < nInputPartitions.length; ++i) {
+                    nInputPartitions[i] = nPartMap.get(jobRun.getJobActivityGraph()
+                            .getProducerActivity(inputs.get(i).getConnectorId()).getOperatorDescriptorId());
+                }
+            }
+            int[] nOutputPartitions = null;
+            List<IConnectorDescriptor> outputs = jobRun.getJobActivityGraph().getActivityOutputConnectorDescriptors(
+                    anId);
+            if (outputs != null) {
+                nOutputPartitions = new int[outputs.size()];
+                for (int i = 0; i < nOutputPartitions.length; ++i) {
+                    nOutputPartitions[i] = nPartMap.get(jobRun.getJobActivityGraph()
+                            .getConsumerActivity(outputs.get(i).getConnectorId()).getOperatorDescriptorId());
+                }
+            }
+            ActivityPartitionDetails apd = new ActivityPartitionDetails(nParts, nInputPartitions, nOutputPartitions);
+            activityPartsMap.put(anId, apd);
+        }
+        return activityPartsMap;
+    }
+
+    public Map<? extends PartitionId, ? extends TaskCluster> getPartitionProducingTaskClusterMap() {
+        return partitionProducingTaskClusterMap;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityPartitionDetails.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityPartitionDetails.java
new file mode 100644
index 0000000..414b543
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityPartitionDetails.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.control.cc.scheduler;
+
+import java.util.Arrays;
+
+public class ActivityPartitionDetails {
+    private final int nPartitions;
+
+    private final int[] nInputPartitions;
+
+    private final int[] nOutputPartitions;
+
+    public ActivityPartitionDetails(int nPartitions, int[] nInputPartitions, int[] nOutputPartitions) {
+        this.nPartitions = nPartitions;
+        this.nInputPartitions = nInputPartitions;
+        this.nOutputPartitions = nOutputPartitions;
+    }
+
+    public int getPartitionCount() {
+        return nPartitions;
+    }
+
+    public int[] getInputPartitionCounts() {
+        return nInputPartitions;
+    }
+
+    public int[] getOutputPartitionCounts() {
+        return nOutputPartitions;
+    }
+
+    @Override
+    public String toString() {
+        return nPartitions + ":" + (nInputPartitions == null ? "[]" : Arrays.toString(nInputPartitions)) + ":"
+                + (nOutputPartitions == null ? "[]" : Arrays.toString(nOutputPartitions));
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobAttemptSchedulerState.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobAttemptSchedulerState.java
deleted file mode 100644
index f8fcb89..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobAttemptSchedulerState.java
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.scheduler;
-
-public interface IJobAttemptSchedulerState {
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
deleted file mode 100644
index 2906ba6..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.scheduler;
-
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-
-public interface ISchedule {
-    public String[] getPartitions(ActivityNodeId aid);
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IScheduler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IScheduler.java
deleted file mode 100644
index 3893a2d..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IScheduler.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.scheduler;
-
-import java.util.Set;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobStageAttempt;
-
-public interface IScheduler {
-    public IJobAttemptSchedulerState createJobAttemptState(JobAttempt ja);
-
-    public void schedule(Set<JobStageAttempt> runnableStageAttempts) throws HyracksException;
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
new file mode 100644
index 0000000..2fc9624
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
@@ -0,0 +1,664 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.scheduler;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.Executor;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+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.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.IConnectorDescriptorVisitor;
+import edu.uci.ics.hyracks.control.cc.job.IOperatorDescriptorVisitor;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.job.PlanUtils;
+import edu.uci.ics.hyracks.control.cc.job.Task;
+import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
+import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
+import edu.uci.ics.hyracks.control.cc.job.TaskClusterAttempt;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.JobCleanupEvent;
+import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
+import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
+
+public class JobScheduler {
+    private static final Logger LOGGER = Logger.getLogger(JobScheduler.class.getName());
+
+    private final ClusterControllerService ccs;
+
+    private final JobRun jobRun;
+
+    private final PartitionConstraintSolver solver;
+
+    private final Map<PartitionId, TaskCluster> partitionProducingTaskClusterMap;
+
+    private final Set<TaskCluster> inProgressTaskClusters;
+
+    private Set<ActivityCluster> rootActivityClusters;
+
+    public JobScheduler(ClusterControllerService ccs, JobRun jobRun) {
+        this.ccs = ccs;
+        this.jobRun = jobRun;
+        solver = new PartitionConstraintSolver();
+        partitionProducingTaskClusterMap = new HashMap<PartitionId, TaskCluster>();
+        inProgressTaskClusters = new HashSet<TaskCluster>();
+    }
+
+    public JobRun getJobRun() {
+        return jobRun;
+    }
+
+    public PartitionConstraintSolver getSolver() {
+        return solver;
+    }
+
+    public void startJob() throws HyracksException {
+        analyze();
+        startRunnableActivityClusters();
+    }
+
+    private void analyze() throws HyracksException {
+        final JobActivityGraph jag = jobRun.getJobActivityGraph();
+        final ICCApplicationContext appCtx = ccs.getApplicationMap().get(jag.getApplicationName());
+        JobSpecification spec = jag.getJobSpecification();
+        final Set<Constraint> contributedConstraints = new HashSet<Constraint>();
+        final IConstraintAcceptor acceptor = new IConstraintAcceptor() {
+            @Override
+            public void addConstraint(Constraint constraint) {
+                contributedConstraints.add(constraint);
+            }
+        };
+        PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
+            @Override
+            public void visit(IOperatorDescriptor op) {
+                op.contributeSchedulingConstraints(acceptor, jag, appCtx);
+            }
+        });
+        PlanUtils.visit(spec, new IConnectorDescriptorVisitor() {
+            @Override
+            public void visit(IConnectorDescriptor conn) {
+                conn.contributeSchedulingConstraints(acceptor, jag, appCtx);
+            }
+        });
+        contributedConstraints.addAll(spec.getUserConstraints());
+        solver.addConstraints(contributedConstraints);
+
+        ActivityClusterGraphBuilder acgb = new ActivityClusterGraphBuilder(jobRun);
+        rootActivityClusters = acgb.inferActivityClusters(jag);
+    }
+
+    private void findRunnableTaskClusterRoots(Set<TaskCluster> frontier, Set<ActivityCluster> roots)
+            throws HyracksException {
+        for (ActivityCluster root : roots) {
+            findRunnableTaskClusterRoots(frontier, root);
+        }
+    }
+
+    private void findRunnableTaskClusterRoots(Set<TaskCluster> frontier, ActivityCluster candidate)
+            throws HyracksException {
+        boolean depsComplete = true;
+        for (ActivityCluster depAC : candidate.getDependencies()) {
+            if (!isPlanned(depAC)) {
+                depsComplete = false;
+                findRunnableTaskClusterRoots(frontier, depAC);
+            } else {
+                boolean tcRootsComplete = true;
+                Set<TaskCluster> depACTCRoots = new HashSet<TaskCluster>();
+                for (TaskCluster tc : depAC.getPlan().getTaskClusters()) {
+                    if (tc.getProducedPartitions().isEmpty()) {
+                        TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
+                        if (tca == null || tca.getStatus() != TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
+                            tcRootsComplete = false;
+                        }
+                        depACTCRoots.add(tc);
+                    }
+                }
+                if (!tcRootsComplete) {
+                    depsComplete = false;
+                    findRunnableTaskClusterRoots(frontier, depAC);
+                }
+            }
+        }
+        if (depsComplete) {
+            if (!isPlanned(candidate)) {
+                ActivityClusterPlanner acp = new ActivityClusterPlanner(this);
+                acp.planActivityCluster(candidate);
+                partitionProducingTaskClusterMap.putAll(acp.getPartitionProducingTaskClusterMap());
+            }
+            for (TaskCluster tc : candidate.getPlan().getTaskClusters()) {
+                if (tc.getProducedPartitions().isEmpty()) {
+                    TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
+                    if (tca == null || tca.getStatus() != TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
+                        frontier.add(tc);
+                    }
+                }
+            }
+        }
+    }
+
+    private boolean isPlanned(ActivityCluster ac) {
+        return ac.getPlan() != null;
+    }
+
+    private void startRunnableActivityClusters() throws HyracksException {
+        Set<TaskCluster> taskClusterRoots = new HashSet<TaskCluster>();
+        findRunnableTaskClusterRoots(taskClusterRoots, rootActivityClusters);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Runnable TC roots: " + taskClusterRoots + ", inProgressTaskClusters: "
+                    + inProgressTaskClusters);
+        }
+        if (taskClusterRoots.isEmpty() && inProgressTaskClusters.isEmpty()) {
+            ccs.getJobQueue().schedule(new JobCleanupEvent(ccs, jobRun.getJobId(), JobStatus.TERMINATED, null));
+            return;
+        }
+        startRunnableTaskClusters(taskClusterRoots);
+    }
+
+    private void startRunnableTaskClusters(Set<TaskCluster> tcRoots) throws HyracksException {
+        Map<TaskCluster, Runnability> runnabilityMap = new HashMap<TaskCluster, Runnability>();
+        for (TaskCluster tc : tcRoots) {
+            assignRunnabilityRank(tc, runnabilityMap);
+        }
+
+        PriorityQueue<RankedRunnableTaskCluster> queue = new PriorityQueue<RankedRunnableTaskCluster>();
+        for (Map.Entry<TaskCluster, Runnability> e : runnabilityMap.entrySet()) {
+            TaskCluster tc = e.getKey();
+            Runnability runnability = e.getValue();
+            if (runnability.getTag() != Runnability.Tag.RUNNABLE) {
+                continue;
+            }
+            int priority = runnability.getPriority();
+            if (priority >= 0 && priority < Integer.MAX_VALUE) {
+                queue.add(new RankedRunnableTaskCluster(priority, tc));
+            }
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Ranked TCs: " + queue);
+        }
+
+        Map<String, List<TaskAttemptDescriptor>> taskAttemptMap = new HashMap<String, List<TaskAttemptDescriptor>>();
+        for (RankedRunnableTaskCluster rrtc : queue) {
+            TaskCluster tc = rrtc.getTaskCluster();
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Found runnable TC: " + tc);
+                List<TaskClusterAttempt> attempts = tc.getAttempts();
+                LOGGER.info("Attempts so far:" + attempts.size());
+                for (TaskClusterAttempt tcAttempt : attempts) {
+                    LOGGER.info("Status: " + tcAttempt.getStatus());
+                }
+            }
+            assignTaskLocations(tc, taskAttemptMap);
+        }
+
+        if (taskAttemptMap.isEmpty()) {
+            return;
+        }
+
+        startTasks(taskAttemptMap);
+    }
+
+    /*
+     * Runnability rank has the following semantics
+     * Runnability(Runnable TaskCluster depending on completed TaskClusters) = {RUNNABLE, 0}
+     * Runnability(Runnable TaskCluster) = max(Rank(Dependent TaskClusters)) + 1
+     * Runnability(Non-schedulable TaskCluster) = {NOT_RUNNABLE, _} 
+     */
+    private Runnability assignRunnabilityRank(TaskCluster goal, Map<TaskCluster, Runnability> runnabilityMap) {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Computing runnability: " + goal);
+        }
+        if (runnabilityMap.containsKey(goal)) {
+            return runnabilityMap.get(goal);
+        }
+        TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(goal);
+        if (lastAttempt != null) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Last Attempt Status: " + lastAttempt.getStatus());
+            }
+            if (lastAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
+                Runnability runnability = new Runnability(Runnability.Tag.COMPLETED, Integer.MIN_VALUE);
+                runnabilityMap.put(goal, runnability);
+                return runnability;
+            }
+            if (lastAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.RUNNING) {
+                Runnability runnability = new Runnability(Runnability.Tag.RUNNING, Integer.MIN_VALUE);
+                runnabilityMap.put(goal, runnability);
+                return runnability;
+            }
+        }
+        Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicyMap = jobRun.getConnectorPolicyMap();
+        PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
+        Runnability aggregateRunnability = new Runnability(Runnability.Tag.RUNNABLE, 0);
+        for (PartitionId pid : goal.getRequiredPartitions()) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Inspecting required partition: " + pid);
+            }
+            Runnability runnability;
+            ConnectorDescriptorId cdId = pid.getConnectorDescriptorId();
+            IConnectorPolicy cPolicy = connectorPolicyMap.get(cdId);
+            PartitionState maxState = pmm.getMaximumAvailableState(pid);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Policy: " + cPolicy + " maxState: " + maxState);
+            }
+            if (PartitionState.COMMITTED.equals(maxState)) {
+                runnability = new Runnability(Runnability.Tag.RUNNABLE, 0);
+            } else if (PartitionState.STARTED.equals(maxState) && !cPolicy.consumerWaitsForProducerToFinish()) {
+                runnability = new Runnability(Runnability.Tag.RUNNABLE, 1);
+            } else {
+                runnability = assignRunnabilityRank(partitionProducingTaskClusterMap.get(pid), runnabilityMap);
+                switch (runnability.getTag()) {
+                    case RUNNABLE:
+                        if (cPolicy.consumerWaitsForProducerToFinish()) {
+                            runnability = new Runnability(Runnability.Tag.NOT_RUNNABLE, Integer.MAX_VALUE);
+                        } else {
+                            runnability = new Runnability(Runnability.Tag.RUNNABLE, runnability.getPriority() + 1);
+                        }
+                        break;
+
+                    case NOT_RUNNABLE:
+                        break;
+
+                    case RUNNING:
+                        if (cPolicy.consumerWaitsForProducerToFinish()) {
+                            runnability = new Runnability(Runnability.Tag.NOT_RUNNABLE, Integer.MAX_VALUE);
+                        } else {
+                            runnability = new Runnability(Runnability.Tag.RUNNABLE, 1);
+                        }
+                        break;
+                }
+            }
+            aggregateRunnability = Runnability.getWorstCase(aggregateRunnability, runnability);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("aggregateRunnability: " + aggregateRunnability);
+            }
+        }
+        runnabilityMap.put(goal, aggregateRunnability);
+        return aggregateRunnability;
+    }
+
+    private void assignTaskLocations(TaskCluster tc, Map<String, List<TaskAttemptDescriptor>> taskAttemptMap)
+            throws HyracksException {
+        JobActivityGraph jag = jobRun.getJobActivityGraph();
+        Task[] tasks = tc.getTasks();
+        List<TaskClusterAttempt> tcAttempts = tc.getAttempts();
+        int attempts = tcAttempts.size();
+        TaskClusterAttempt tcAttempt = new TaskClusterAttempt(tc, attempts);
+        TaskAttempt[] taskAttempts = new TaskAttempt[tasks.length];
+        Map<TaskId, LValueConstraintExpression> locationMap = new HashMap<TaskId, LValueConstraintExpression>();
+        for (int i = 0; i < tasks.length; ++i) {
+            Task ts = tasks[i];
+            TaskId tid = ts.getTaskId();
+            TaskAttempt taskAttempt = new TaskAttempt(tcAttempt, new TaskAttemptId(new TaskId(tid.getActivityId(),
+                    tid.getPartition()), attempts), ts);
+            taskAttempt.setStatus(TaskAttempt.TaskStatus.INITIALIZED, null);
+            locationMap.put(tid,
+                    new PartitionLocationExpression(tid.getActivityId().getOperatorDescriptorId(), tid.getPartition()));
+            taskAttempts[i] = taskAttempt;
+        }
+        tcAttempt.setTaskAttempts(taskAttempts);
+        solver.solve(locationMap.values());
+        for (int i = 0; i < tasks.length; ++i) {
+            Task ts = tasks[i];
+            TaskId tid = ts.getTaskId();
+            TaskAttempt taskAttempt = taskAttempts[i];
+            ActivityId aid = tid.getActivityId();
+            Set<ActivityId> blockers = jag.getBlocked2BlockerMap().get(aid);
+            String nodeId = null;
+            if (blockers != null) {
+                for (ActivityId blocker : blockers) {
+                    nodeId = findLocationOfBlocker(jobRun, jag, new TaskId(blocker, tid.getPartition()));
+                    if (nodeId != null) {
+                        break;
+                    }
+                }
+            }
+            Set<String> liveNodes = ccs.getNodeMap().keySet();
+            if (nodeId == null) {
+                LValueConstraintExpression pLocationExpr = locationMap.get(tid);
+                Object location = solver.getValue(pLocationExpr);
+                if (location == null) {
+                    // pick any
+                    nodeId = liveNodes.toArray(new String[liveNodes.size()])[Math.abs(new Random().nextInt())
+                            % liveNodes.size()];
+                } else if (location instanceof String) {
+                    nodeId = (String) location;
+                } else if (location instanceof String[]) {
+                    for (String choice : (String[]) location) {
+                        if (liveNodes.contains(choice)) {
+                            nodeId = choice;
+                            break;
+                        }
+                    }
+                    if (nodeId == null) {
+                        throw new HyracksException("No satisfiable location found for "
+                                + taskAttempt.getTaskAttemptId());
+                    }
+                } else {
+                    throw new HyracksException("Unknown type of value for " + pLocationExpr + ": " + location + "("
+                            + location.getClass() + ")");
+                }
+            }
+            if (nodeId == null) {
+                throw new HyracksException("No satisfiable location found for " + taskAttempt.getTaskAttemptId());
+            }
+            if (!liveNodes.contains(nodeId)) {
+                throw new HyracksException("Node " + nodeId + " not live");
+            }
+            taskAttempt.setNodeId(nodeId);
+            taskAttempt.setStatus(TaskAttempt.TaskStatus.RUNNING, null);
+            List<TaskAttemptDescriptor> tads = taskAttemptMap.get(nodeId);
+            if (tads == null) {
+                tads = new ArrayList<TaskAttemptDescriptor>();
+                taskAttemptMap.put(nodeId, tads);
+            }
+            ActivityPartitionDetails apd = ts.getActivityPlan().getActivityPartitionDetails();
+            tads.add(new TaskAttemptDescriptor(taskAttempt.getTaskAttemptId(), apd.getPartitionCount(), apd
+                    .getInputPartitionCounts(), apd.getOutputPartitionCounts()));
+        }
+        tcAttempt.initializePendingTaskCounter();
+        tcAttempts.add(tcAttempt);
+        tcAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.RUNNING);
+        inProgressTaskClusters.add(tc);
+    }
+
+    private static String findLocationOfBlocker(JobRun jobRun, JobActivityGraph jag, TaskId tid) {
+        ActivityId blockerAID = tid.getActivityId();
+        ActivityCluster blockerAC = jobRun.getActivityClusterMap().get(blockerAID);
+        Task[] blockerTasks = blockerAC.getPlan().getActivityPlanMap().get(blockerAID).getTasks();
+        List<TaskClusterAttempt> tcAttempts = blockerTasks[tid.getPartition()].getTaskCluster().getAttempts();
+        if (tcAttempts == null || tcAttempts.isEmpty()) {
+            return null;
+        }
+        TaskClusterAttempt lastTCA = tcAttempts.get(tcAttempts.size() - 1);
+        for (TaskAttempt ta : lastTCA.getTaskAttempts()) {
+            TaskId blockerTID = ta.getTaskAttemptId().getTaskId();
+            if (tid.equals(blockerTID)) {
+                return ta.getNodeId();
+            }
+        }
+        return null;
+    }
+
+    private static TaskClusterAttempt findLastTaskClusterAttempt(TaskCluster tc) {
+        List<TaskClusterAttempt> attempts = tc.getAttempts();
+        if (!attempts.isEmpty()) {
+            return attempts.get(attempts.size() - 1);
+        }
+        return null;
+    }
+
+    private void startTasks(Map<String, List<TaskAttemptDescriptor>> taskAttemptMap) throws HyracksException {
+        Executor executor = ccs.getExecutor();
+        final JobId jobId = jobRun.getJobId();
+        final JobActivityGraph jag = jobRun.getJobActivityGraph();
+        final String appName = jag.getApplicationName();
+        final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = jobRun.getConnectorPolicyMap();
+        for (Map.Entry<String, List<TaskAttemptDescriptor>> e : taskAttemptMap.entrySet()) {
+            String nodeId = e.getKey();
+            final List<TaskAttemptDescriptor> taskDescriptors = e.getValue();
+            final NodeControllerState node = ccs.getNodeMap().get(nodeId);
+            if (node != null) {
+                node.getActiveJobIds().add(jobRun.getJobId());
+                jobRun.getParticipatingNodeIds().add(nodeId);
+                executor.execute(new Runnable() {
+                    @Override
+                    public void run() {
+                        try {
+                            node.getNodeController().startTasks(appName, jobId, JavaSerializationUtils.serialize(jag),
+                                    taskDescriptors, connectorPolicies, null);
+                        } catch (IOException e) {
+                            e.printStackTrace();
+                        } catch (Exception e) {
+                            e.printStackTrace();
+                        }
+                    }
+                });
+            }
+        }
+    }
+
+    private void abortJob(Exception exception) {
+        for (TaskCluster tc : inProgressTaskClusters) {
+            abortTaskCluster(findLastTaskClusterAttempt(tc));
+        }
+        jobRun.setStatus(JobStatus.FAILURE, exception);
+    }
+
+    private void abortTaskCluster(TaskClusterAttempt tcAttempt) {
+        Set<TaskAttemptId> abortTaskIds = new HashSet<TaskAttemptId>();
+        Map<String, List<TaskAttemptId>> abortTaskAttemptMap = new HashMap<String, List<TaskAttemptId>>();
+        for (TaskAttempt ta : tcAttempt.getTaskAttempts()) {
+            TaskAttemptId taId = ta.getTaskAttemptId();
+            abortTaskIds.add(taId);
+            if (ta.getStatus() == TaskAttempt.TaskStatus.RUNNING) {
+                ta.setStatus(TaskAttempt.TaskStatus.ABORTED, null);
+                List<TaskAttemptId> abortTaskAttempts = abortTaskAttemptMap.get(ta.getNodeId());
+                if (abortTaskAttempts == null) {
+                    abortTaskAttempts = new ArrayList<TaskAttemptId>();
+                    abortTaskAttemptMap.put(ta.getNodeId(), abortTaskAttempts);
+                }
+                abortTaskAttempts.add(taId);
+            }
+        }
+        final JobId jobId = jobRun.getJobId();
+        for (Map.Entry<String, List<TaskAttemptId>> e : abortTaskAttemptMap.entrySet()) {
+            final NodeControllerState node = ccs.getNodeMap().get(e.getKey());
+            final List<TaskAttemptId> abortTaskAttempts = e.getValue();
+            if (node != null) {
+                LOGGER.info("Aborting: " + abortTaskAttempts + " at " + e.getKey());
+                ccs.getExecutor().execute(new Runnable() {
+                    @Override
+                    public void run() {
+                        try {
+                            node.getNodeController().abortTasks(jobId, abortTaskAttempts);
+                        } catch (Exception e) {
+                            e.printStackTrace();
+                        }
+                    }
+                });
+            }
+        }
+        inProgressTaskClusters.remove(tcAttempt.getTaskCluster());
+        TaskCluster tc = tcAttempt.getTaskCluster();
+        PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
+        pmm.removeUncommittedPartitions(tc.getProducedPartitions(), abortTaskIds);
+        pmm.removePartitionRequests(tc.getRequiredPartitions(), abortTaskIds);
+    }
+
+    private void abortDoomedTaskClusters() throws HyracksException {
+        Set<TaskCluster> doomedTaskClusters = new HashSet<TaskCluster>();
+        for (TaskCluster tc : inProgressTaskClusters) {
+            // Start search at TCs that produce no outputs (sinks)
+            if (tc.getProducedPartitions().isEmpty()) {
+                findDoomedTaskClusters(tc, doomedTaskClusters);
+            }
+        }
+
+        for (TaskCluster tc : doomedTaskClusters) {
+            TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
+            abortTaskCluster(tca);
+            tca.setStatus(TaskClusterAttempt.TaskClusterStatus.ABORTED);
+        }
+    }
+
+    private boolean findDoomedTaskClusters(TaskCluster tc, Set<TaskCluster> doomedTaskClusters) {
+        if (doomedTaskClusters.contains(tc)) {
+            return true;
+        }
+        TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
+        if (lastAttempt != null) {
+            switch (lastAttempt.getStatus()) {
+                case ABORTED:
+                case FAILED:
+                    return true;
+
+                case COMPLETED:
+                    return false;
+            }
+        }
+        Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicyMap = jobRun.getConnectorPolicyMap();
+        PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
+        boolean doomed = false;
+        for (TaskCluster depTC : tc.getDependencyTaskClusters()) {
+            if (findDoomedTaskClusters(depTC, doomedTaskClusters)) {
+                doomed = true;
+            }
+        }
+        for (PartitionId pid : tc.getRequiredPartitions()) {
+            ConnectorDescriptorId cdId = pid.getConnectorDescriptorId();
+            IConnectorPolicy cPolicy = connectorPolicyMap.get(cdId);
+            PartitionState maxState = pmm.getMaximumAvailableState(pid);
+            if (maxState == null
+                    || (cPolicy.consumerWaitsForProducerToFinish() && maxState != PartitionState.COMMITTED)) {
+                if (findDoomedTaskClusters(partitionProducingTaskClusterMap.get(pid), doomedTaskClusters)) {
+                    doomed = true;
+                }
+            }
+        }
+        if (doomed) {
+            doomedTaskClusters.add(tc);
+        }
+        return doomed;
+    }
+
+    public void notifyTaskComplete(TaskAttempt ta) throws HyracksException {
+        TaskAttemptId taId = ta.getTaskAttemptId();
+        TaskCluster tc = ta.getTaskState().getTaskCluster();
+        TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
+        if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
+            TaskAttempt.TaskStatus taStatus = ta.getStatus();
+            if (taStatus == TaskAttempt.TaskStatus.RUNNING) {
+                ta.setStatus(TaskAttempt.TaskStatus.COMPLETED, null);
+                if (lastAttempt.decrementPendingTasksCounter() == 0) {
+                    lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.COMPLETED);
+                    inProgressTaskClusters.remove(tc);
+                    startRunnableActivityClusters();
+                }
+            } else {
+                LOGGER.warning("Spurious task complete notification: " + taId + " Current state = " + taStatus);
+            }
+        } else {
+            LOGGER.warning("Ignoring task complete notification: " + taId + " -- Current last attempt = " + lastAttempt);
+        }
+    }
+
+    /**
+     * Indicates that a single task attempt has encountered a failure.
+     * 
+     * @param ta
+     *            - Failed Task Attempt
+     * @param ac
+     *            - Activity Cluster that owns this Task
+     * @param exception
+     *            - Cause of the failure
+     */
+    public void notifyTaskFailure(TaskAttempt ta, ActivityCluster ac, Exception exception) {
+        try {
+            TaskAttemptId taId = ta.getTaskAttemptId();
+            TaskCluster tc = ta.getTaskState().getTaskCluster();
+            TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
+            if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
+                TaskAttempt.TaskStatus taStatus = ta.getStatus();
+                if (taStatus == TaskAttempt.TaskStatus.RUNNING) {
+                    ta.setStatus(TaskAttempt.TaskStatus.FAILED, exception);
+                    abortTaskCluster(lastAttempt);
+                    lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.FAILED);
+                    abortDoomedTaskClusters();
+                    if (lastAttempt.getAttempt() >= ac.getMaxTaskClusterAttempts()) {
+                        abortJob(null);
+                        return;
+                    }
+                    startRunnableActivityClusters();
+                } else {
+                    LOGGER.warning("Spurious task failure notification: " + taId + " Current state = " + taStatus);
+                }
+            } else {
+                LOGGER.warning("Ignoring task failure notification: " + taId + " -- Current last attempt = "
+                        + lastAttempt);
+            }
+        } catch (Exception e) {
+            abortJob(e);
+        }
+    }
+
+    /**
+     * Indicates that the provided set of nodes have left the cluster.
+     * 
+     * @param deadNodes
+     *            - Set of failed nodes
+     */
+    public void notifyNodeFailures(Set<String> deadNodes) {
+        try {
+            jobRun.getPartitionMatchMaker().notifyNodeFailures(deadNodes);
+            for (ActivityCluster ac : jobRun.getActivityClusters()) {
+                TaskCluster[] taskClusters = ac.getPlan().getTaskClusters();
+                if (taskClusters != null) {
+                    for (TaskCluster tc : taskClusters) {
+                        TaskClusterAttempt lastTaskClusterAttempt = findLastTaskClusterAttempt(tc);
+                        if (lastTaskClusterAttempt != null
+                                && (lastTaskClusterAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.COMPLETED || lastTaskClusterAttempt
+                                        .getStatus() == TaskClusterAttempt.TaskClusterStatus.RUNNING)) {
+                            boolean abort = false;
+                            for (TaskAttempt ta : lastTaskClusterAttempt.getTaskAttempts()) {
+                                assert (ta.getStatus() == TaskAttempt.TaskStatus.COMPLETED || ta.getStatus() == TaskAttempt.TaskStatus.RUNNING);
+                                if (deadNodes.contains(ta.getNodeId())) {
+                                    ta.setStatus(TaskAttempt.TaskStatus.FAILED,
+                                            new HyracksException("Node " + ta.getNodeId() + " failed"));
+                                    abort = true;
+                                }
+                            }
+                            if (abort) {
+                                abortTaskCluster(lastTaskClusterAttempt);
+                            }
+                        }
+                    }
+                    abortDoomedTaskClusters();
+                }
+            }
+            startRunnableActivityClusters();
+        } catch (Exception e) {
+            abortJob(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/PartitionConstraintSolver.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/PartitionConstraintSolver.java
new file mode 100644
index 0000000..ea10562
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/PartitionConstraintSolver.java
@@ -0,0 +1,125 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.scheduler;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+
+public class PartitionConstraintSolver {
+    private final Map<LValueConstraintExpression, Set<ConstraintExpression>> constraints;
+
+    public PartitionConstraintSolver() {
+        constraints = new HashMap<LValueConstraintExpression, Set<ConstraintExpression>>();
+    }
+
+    public void addConstraints(Collection<Constraint> constraintCollection) {
+        for (Constraint c : constraintCollection) {
+            addConstraint(c);
+        }
+    }
+
+    public void addConstraint(Constraint c) {
+        Set<ConstraintExpression> rValues = constraints.get(c.getLValue());
+        if (rValues == null) {
+            rValues = new HashSet<ConstraintExpression>();
+            constraints.put(c.getLValue(), rValues);
+        }
+        rValues.add(c.getRValue());
+    }
+
+    public void solve(Collection<LValueConstraintExpression> targetSet) {
+        Set<LValueConstraintExpression> inProcess = new HashSet<LValueConstraintExpression>();
+        for (LValueConstraintExpression lv : targetSet) {
+            solveLValue(lv, inProcess);
+        }
+    }
+
+    private Solution solve(ConstraintExpression ce, Set<LValueConstraintExpression> inProcess) {
+        switch (ce.getTag()) {
+            case CONSTANT:
+                return new Solution(((ConstantExpression) ce).getValue(), Solution.Status.FOUND);
+
+            case PARTITION_COUNT:
+            case PARTITION_LOCATION:
+                return solveLValue((LValueConstraintExpression) ce, inProcess);
+        }
+        return null;
+    }
+
+    private Solution solveLValue(LValueConstraintExpression lv, Set<LValueConstraintExpression> inProcess) {
+        if (inProcess.contains(lv)) {
+            return new Solution(null, Solution.Status.CYCLE);
+        }
+        Solution result = null;
+        inProcess.add(lv);
+        Set<ConstraintExpression> rValues = constraints.get(lv);
+        if (rValues == null) {
+            return new Solution(null, Solution.Status.NOT_BOUND);
+        }
+        for (ConstraintExpression ce : rValues) {
+            Solution solution = solve(ce, inProcess);
+            if (solution != null && solution.status == Solution.Status.FOUND) {
+                result = solution;
+                break;
+            }
+        }
+        if (result != null) {
+            rValues.clear();
+            rValues.add(new ConstantExpression(result.value));
+        }
+        inProcess.remove(lv);
+        return result;
+    }
+
+    public Object getValue(LValueConstraintExpression lValue) {
+        Set<ConstraintExpression> rValues = constraints.get(lValue);
+        if (rValues == null) {
+            return null;
+        }
+        if (rValues.size() != 1) {
+            return null;
+        }
+        for (ConstraintExpression ce : rValues) {
+            if (ce.getTag() == ConstraintExpression.ExpressionTag.CONSTANT) {
+                return ((ConstantExpression) ce).getValue();
+            }
+        }
+        return null;
+    }
+
+    private static class Solution {
+        enum Status {
+            FOUND,
+            CYCLE,
+            NOT_BOUND,
+        }
+
+        final Object value;
+        final Status status;
+
+        public Solution(Object value, Status status) {
+            this.value = value;
+            this.status = status;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/RankedRunnableTaskCluster.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/RankedRunnableTaskCluster.java
new file mode 100644
index 0000000..27e9480
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/RankedRunnableTaskCluster.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.scheduler;
+
+import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
+
+public class RankedRunnableTaskCluster implements Comparable<RankedRunnableTaskCluster> {
+    private final int rank;
+
+    private final TaskCluster taskCluster;
+
+    public RankedRunnableTaskCluster(int rank, TaskCluster taskCluster) {
+        this.rank = rank;
+        this.taskCluster = taskCluster;
+    }
+
+    public int getRank() {
+        return rank;
+    }
+
+    public TaskCluster getTaskCluster() {
+        return taskCluster;
+    }
+
+    @Override
+    public String toString() {
+        return "[" + rank + ":" + taskCluster + "]";
+    }
+
+    @Override
+    public int compareTo(RankedRunnableTaskCluster o) {
+        int cmp = rank - o.rank;
+        return cmp < 0 ? -1 : (cmp > 0 ? 1 : 0);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/Runnability.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/Runnability.java
new file mode 100644
index 0000000..56ae9c3
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/Runnability.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.control.cc.scheduler;
+
+public final class Runnability {
+    private final Tag tag;
+
+    private final int priority;
+
+    public Runnability(Tag tag, int priority) {
+        this.tag = tag;
+        this.priority = priority;
+    }
+
+    public Tag getTag() {
+        return tag;
+    }
+
+    public int getPriority() {
+        return priority;
+    }
+
+    public enum Tag {
+        COMPLETED,
+        NOT_RUNNABLE,
+        RUNNABLE,
+        RUNNING,
+    }
+
+    public static Runnability getWorstCase(Runnability r1, Runnability r2) {
+        switch (r1.tag) {
+            case COMPLETED:
+                switch (r2.tag) {
+                    case COMPLETED:
+                    case NOT_RUNNABLE:
+                    case RUNNABLE:
+                    case RUNNING:
+                        return r2;
+                }
+                break;
+
+            case NOT_RUNNABLE:
+                switch (r2.tag) {
+                    case COMPLETED:
+                    case NOT_RUNNABLE:
+                    case RUNNABLE:
+                    case RUNNING:
+                        return r1;
+                }
+                break;
+
+            case RUNNABLE:
+                switch (r2.tag) {
+                    case COMPLETED:
+                        return r1;
+
+                    case RUNNING:
+                        return r1.priority > 0 ? r1 : new Runnability(Tag.RUNNABLE, 1);
+
+                    case NOT_RUNNABLE:
+                        return r2;
+
+                    case RUNNABLE:
+                        return r1.priority > r2.priority ? r1 : r2;
+                }
+                break;
+
+            case RUNNING:
+                switch (r2.tag) {
+                    case COMPLETED:
+                    case RUNNING:
+                        return r1;
+
+                    case NOT_RUNNABLE:
+                        return r2;
+
+                    case RUNNABLE:
+                        return r2.priority > 0 ? r2 : new Runnability(Tag.RUNNABLE, 1);
+                }
+                break;
+        }
+        throw new IllegalArgumentException("Could not aggregate: " + r1 + " and " + r2);
+    }
+
+    @Override
+    public String toString() {
+        return "{" + tag + ", " + priority + "}";
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/naive/NaiveScheduler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/naive/NaiveScheduler.java
deleted file mode 100644
index c433bf9..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/naive/NaiveScheduler.java
+++ /dev/null
@@ -1,208 +0,0 @@
-package edu.uci.ics.hyracks.control.cc.scheduler.naive;
-
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.constraints.expressions.BelongsToExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression.ExpressionTag;
-import edu.uci.ics.hyracks.api.constraints.expressions.EnumeratedCollectionExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.RelationalExpression;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobStageAttempt;
-import edu.uci.ics.hyracks.control.cc.scheduler.IJobAttemptSchedulerState;
-import edu.uci.ics.hyracks.control.cc.scheduler.ISchedule;
-import edu.uci.ics.hyracks.control.cc.scheduler.IScheduler;
-
-public class NaiveScheduler implements IScheduler {
-    private static final Logger LOGGER = Logger.getLogger(NaiveScheduler.class.getName());
-
-    private final ClusterControllerService ccs;
-
-    public NaiveScheduler(ClusterControllerService ccs) {
-        this.ccs = ccs;
-    }
-
-    @Override
-    public IJobAttemptSchedulerState createJobAttemptState(JobAttempt ja) {
-        return new JobAttemptState(ja.getJobRun().getConstraints());
-    }
-
-    @Override
-    public void schedule(Set<JobStageAttempt> runnableStageAttempts) throws HyracksException {
-        for (JobStageAttempt jsa : runnableStageAttempts) {
-            Set<OperatorDescriptorId> operators = new HashSet<OperatorDescriptorId>();
-            for (ActivityNodeId aid : jsa.getJobStage().getTasks()) {
-                operators.add(aid.getOperatorDescriptorId());
-            }
-            jsa.setSchedule(computeSchedule(jsa, operators));
-        }
-    }
-
-    private ISchedule computeSchedule(JobStageAttempt jsa, Set<OperatorDescriptorId> operators) throws HyracksException {
-        Set<String> nodeSet = ccs.getNodeMap().keySet();
-        if (nodeSet.isEmpty()) {
-            throw new HyracksException("0 usable nodes found");
-        }
-        String[] liveNodes = ccs.getNodeMap().keySet().toArray(new String[nodeSet.size()]);
-        JobAttempt ja = jsa.getJobAttempt();
-        final JobAttemptState jas = (JobAttemptState) ja.getSchedulerState();
-
-        List<PartitionLocationExpression> rrAssignment = new ArrayList<PartitionLocationExpression>();
-
-        for (OperatorDescriptorId oid : operators) {
-            String[] opParts = null;
-            if (!jas.allocations.containsKey(oid)) {
-                Set<ConstraintExpression> opConstraints = jas.opConstraints.get(oid);
-                for (ConstraintExpression ce : opConstraints) {
-                    int nParts = getNumPartitions(oid, ce);
-                    if (nParts != -1) {
-                        opParts = new String[nParts];
-                        break;
-                    }
-                }
-                if (opParts == null) {
-                    throw new HyracksException("Unable to satisfy constraints for operator : " + oid);
-                }
-                jas.allocations.put(oid, opParts);
-                BitSet unassignedPartsIds = new BitSet(opParts.length);
-                unassignedPartsIds.set(0, opParts.length);
-                for (ConstraintExpression ce : opConstraints) {
-                    if (ce.getTag() == ConstraintExpression.ExpressionTag.BELONGS_TO) {
-                        BelongsToExpression bE = (BelongsToExpression) ce;
-                        if (bE.getItemExpression().getTag() == ConstraintExpression.ExpressionTag.PARTITION_LOCATION) {
-                            PartitionLocationExpression plE = (PartitionLocationExpression) bE.getItemExpression();
-                            if (plE.getOperatorDescriptorId().equals(oid)) {
-                                int part = plE.getPartition();
-                                if (bE.getSetExpression().getTag() == ConstraintExpression.ExpressionTag.ENUMERATED_SET) {
-                                    EnumeratedCollectionExpression ecE = (EnumeratedCollectionExpression) bE
-                                            .getSetExpression();
-                                    for (ConstraintExpression value : ecE.getMembers()) {
-                                        if (value.getTag() == ConstraintExpression.ExpressionTag.CONSTANT) {
-                                            ConstantExpression nodeConst = (ConstantExpression) value;
-                                            String nodeId = (String) nodeConst.getValue();
-                                            if (nodeSet.contains(nodeId)) {
-                                                opParts[part] = nodeId;
-                                                unassignedPartsIds.clear(part);
-                                                LOGGER.info("Assigned: " + oid + ":" + part + ": " + nodeId);
-                                                break;
-                                            }
-                                        }
-                                    }
-                                }
-                                if (unassignedPartsIds.get(part)) {
-                                    throw new HyracksException("Unsatisfiable constraint for operator: " + oid);
-                                }
-                            }
-                        }
-                    }
-                }
-
-                if (!unassignedPartsIds.isEmpty()) {
-                    // Do round robin assignment.
-                    for (int i = unassignedPartsIds.nextSetBit(0); i >= 0; i = unassignedPartsIds.nextSetBit(i + 1)) {
-                        rrAssignment.add(new PartitionLocationExpression(oid, i));
-                    }
-                }
-            }
-        }
-        int n = rrAssignment.size();
-        for (int i = 0; i < n; ++i) {
-            PartitionLocationExpression plE = rrAssignment.get(i);
-            String[] opParts = jas.allocations.get(plE.getOperatorDescriptorId());
-            String node = liveNodes[i % liveNodes.length];
-            LOGGER.info("Assigned: " + plE.getOperatorDescriptorId() + ":" + plE.getPartition() + ": " + node);
-            opParts[plE.getPartition()] = node;
-        }
-        return new ISchedule() {
-            @Override
-            public String[] getPartitions(ActivityNodeId aid) {
-                return jas.allocations.get(aid.getOperatorDescriptorId());
-            }
-        };
-    }
-
-    private int getNumPartitions(OperatorDescriptorId oid, ConstraintExpression ce) {
-        if (ce.getTag() == ExpressionTag.RELATIONAL) {
-            RelationalExpression re = (RelationalExpression) ce;
-            if (re.getOperator() == RelationalExpression.Operator.EQUAL) {
-                if (re.getLeft().getTag() == ConstraintExpression.ExpressionTag.PARTITION_COUNT) {
-                    return getNumPartitions(oid, (PartitionCountExpression) re.getLeft(), re.getRight());
-                } else if (re.getRight().getTag() == ConstraintExpression.ExpressionTag.PARTITION_COUNT) {
-                    return getNumPartitions(oid, (PartitionCountExpression) re.getRight(), re.getLeft());
-                }
-            }
-        }
-        return -1;
-    }
-
-    private int getNumPartitions(OperatorDescriptorId oid, PartitionCountExpression pce, ConstraintExpression ce) {
-        if (pce.getOperatorDescriptorId().equals(oid)) {
-            if (ce.getTag() == ConstraintExpression.ExpressionTag.CONSTANT) {
-                ConstantExpression constExpr = (ConstantExpression) ce;
-                Integer n = (Integer) constExpr.getValue();
-                return n.intValue();
-            }
-        }
-        return -1;
-    }
-
-    private static class JobAttemptState implements IJobAttemptSchedulerState {
-        final Map<OperatorDescriptorId, String[]> allocations;
-        final Map<OperatorDescriptorId, Set<ConstraintExpression>> opConstraints;
-
-        public JobAttemptState(Set<ConstraintExpression> constraints) {
-            allocations = new HashMap<OperatorDescriptorId, String[]>();
-            opConstraints = new HashMap<OperatorDescriptorId, Set<ConstraintExpression>>();
-            List<ConstraintExpression> ceList = new ArrayList<ConstraintExpression>();
-            for (ConstraintExpression ce : constraints) {
-                ceList.clear();
-                ceList.add(ce);
-                getAllConstraints(ceList);
-                for (ConstraintExpression ce2 : ceList) {
-                    switch (ce2.getTag()) {
-                        case PARTITION_COUNT:
-                            addToOpConstraints(opConstraints,
-                                    ((PartitionCountExpression) ce2).getOperatorDescriptorId(), ce);
-                            break;
-
-                        case PARTITION_LOCATION:
-                            addToOpConstraints(opConstraints,
-                                    ((PartitionLocationExpression) ce2).getOperatorDescriptorId(), ce);
-                            break;
-                    }
-                }
-            }
-        }
-
-        private static void addToOpConstraints(Map<OperatorDescriptorId, Set<ConstraintExpression>> opc,
-                OperatorDescriptorId opId, ConstraintExpression ce) {
-            Set<ConstraintExpression> opSet = opc.get(opId);
-            if (opSet == null) {
-                opSet = new HashSet<ConstraintExpression>();
-                opc.put(opId, opSet);
-            }
-            opSet.add(ce);
-        }
-
-        private static void getAllConstraints(List<ConstraintExpression> ceList) {
-            for (int i = 0; i < ceList.size(); ++i) {
-                ConstraintExpression cE = ceList.get(i);
-                cE.getChildren(ceList);
-            }
-        }
-    }
-}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/AdminConsoleHandler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/AdminConsoleHandler.java
index 1a26e60..a0510f9 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/AdminConsoleHandler.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/AdminConsoleHandler.java
@@ -27,7 +27,7 @@
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 
 public class AdminConsoleHandler extends AbstractHandler {
     private ClusterControllerService ccs;
@@ -51,7 +51,7 @@
         writer.println("<h2>Node Controllers</h2>");
         writer.println("<table><tr><td>Node Id</td><td>Host</td></tr>");
         try {
-            ccs.getJobQueue().scheduleAndSync(new SynchronizableRunnable() {
+            ccs.getJobQueue().scheduleAndSync(new SynchronizableEvent() {
                 @Override
                 protected void doRun() throws Exception {
                     for (Map.Entry<String, NodeControllerState> e : ccs.getNodeMap().entrySet()) {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
index 65c156c..e047212 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
@@ -28,7 +28,7 @@
 import org.eclipse.jetty.server.handler.AbstractHandler;
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
 
 public class ApplicationInstallationHandler extends AbstractHandler {
@@ -54,7 +54,7 @@
             }
             final String appName = parts[0];
             if (HttpMethods.PUT.equals(request.getMethod())) {
-                class OutputStreamGetter extends SynchronizableRunnable {
+                class OutputStreamGetter extends SynchronizableEvent {
                     private OutputStream os;
 
                     @Override
@@ -78,7 +78,7 @@
                     r.os.close();
                 }
             } else if (HttpMethods.GET.equals(request.getMethod())) {
-                class InputStreamGetter extends SynchronizableRunnable {
+                class InputStreamGetter extends SynchronizableEvent {
                     private InputStream is;
 
                     @Override
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/RESTAPIFunction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/RESTAPIFunction.java
index 716dfcb..edb4e93 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/RESTAPIFunction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/RESTAPIFunction.java
@@ -14,11 +14,11 @@
  */
 package edu.uci.ics.hyracks.control.cc.web;
 
-import java.util.UUID;
-
 import org.json.JSONObject;
 
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobActivityGraphJSONEvent;
 import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobProfileJSONEvent;
 import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobSpecificationJSONEvent;
 import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobSummariesJSONEvent;
@@ -47,25 +47,22 @@
             }
 
             case 2: {
-                UUID jobId = UUID.fromString(arguments[0]);
+                JobId jobId = JobId.parse(arguments[0]);
 
-                if ("spec".equalsIgnoreCase(arguments[1])) {
+                if ("job-specification".equalsIgnoreCase(arguments[1])) {
                     GetJobSpecificationJSONEvent gjse = new GetJobSpecificationJSONEvent(ccs, jobId);
                     ccs.getJobQueue().scheduleAndSync(gjse);
-                    result.put("result", gjse.getSpecification());
-                }
-                break;
-            }
-
-            case 3: {
-                UUID jobId = UUID.fromString(arguments[0]);
-                int attempt = Integer.parseInt(arguments[1]);
-
-                if ("profile".equalsIgnoreCase(arguments[2])) {
-                    GetJobProfileJSONEvent gjpe = new GetJobProfileJSONEvent(ccs, jobId, attempt);
+                    result.put("result", gjse.getJSON());
+                } else if ("job-activity-graph".equalsIgnoreCase(arguments[1])) {
+                    GetJobActivityGraphJSONEvent gjage = new GetJobActivityGraphJSONEvent(ccs, jobId);
+                    ccs.getJobQueue().scheduleAndSync(gjage);
+                    result.put("result", gjage.getJSON());
+                } else if ("profile".equalsIgnoreCase(arguments[1])) {
+                    GetJobProfileJSONEvent gjpe = new GetJobProfileJSONEvent(ccs, jobId);
                     ccs.getJobQueue().scheduleAndSync(gjpe);
-                    result.put("result", gjpe.getProfile());
+                    result.put("result", gjpe.getJSON());
                 }
+
                 break;
             }
         }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
index 806a14e..efbc8f7 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
@@ -46,7 +46,7 @@
     }
 
     private void addHandlers() {
-        ContextHandler handler = new ContextHandler("/state");
+        ContextHandler handler = new ContextHandler("/rest");
         RoutingHandler rh = new RoutingHandler();
         rh.addHandler("jobs", new JSONOutputRequestHandler(new RESTAPIFunction(ccs)));
         handler.setHandler(rh);
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
new file mode 100644
index 0000000..6ff4901
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.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.control.common.base;
+
+import java.rmi.Remote;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
+
+public interface IClusterController extends Remote {
+    public NodeParameters registerNode(NodeRegistration reg) throws Exception;
+
+    public void unregisterNode(INodeController nodeController) throws Exception;
+
+    public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
+            throws Exception;
+
+    public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, Exception e) throws Exception;
+
+    public void nodeHeartbeat(String id) throws Exception;
+
+    public void reportProfile(String id, List<JobProfile> profiles) throws Exception;
+
+    public void registerPartitionProvider(PartitionDescriptor partitionDescriptor) throws Exception;
+
+    public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
new file mode 100644
index 0000000..33548d9
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.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.control.common.base;
+
+import java.rmi.Remote;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NodeCapability;
+import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
+
+public interface INodeController extends Remote {
+    public String getId() throws Exception;
+
+    public NCConfig getConfiguration() throws Exception;
+
+    public NodeCapability getNodeCapability() throws Exception;
+
+    public void startTasks(String appName, JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
+            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, byte[] ctxVarBytes) throws Exception;
+
+    public void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception;
+
+    public void cleanUpJob(JobId jobId) throws Exception;
+
+    public void notifyRegistration(IClusterController ccs) throws Exception;
+
+    public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
+            throws Exception;
+
+    public void destroyApplication(String appName) throws Exception;
+
+    public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/CCConfig.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
similarity index 96%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/CCConfig.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
index 4d9b5a0..d12d037 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/CCConfig.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.control.common.controllers;
 
 import org.kohsuke.args4j.Option;
 
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
similarity index 97%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index 4a0f64d..c887223 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.control.common.controllers;
 
 import java.io.Serializable;
 
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeCapability.java
similarity index 94%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeCapability.java
index 88efc1f..7ad5fa1 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeCapability.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.control.common.controllers;
 
 import java.io.Serializable;
 
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeParameters.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeParameters.java
similarity index 96%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeParameters.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeParameters.java
index 8745b32..0161f96 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeParameters.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeParameters.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.control.common.controllers;
 
 import java.io.Serializable;
 
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
new file mode 100644
index 0000000..374cf48
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.common.controllers;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
+
+public final class NodeRegistration implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final INodeController nc;
+
+    private final String nodeId;
+
+    private final NCConfig ncConfig;
+
+    private final NetworkAddress dataPort;
+
+    public NodeRegistration(INodeController nc, String nodeId, NCConfig ncConfig, NetworkAddress dataPort) {
+        this.nc = nc;
+        this.nodeId = nodeId;
+        this.ncConfig = ncConfig;
+        this.dataPort = dataPort;
+    }
+
+    public INodeController getNodeController() {
+        return nc;
+    }
+
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public NCConfig getNCConfig() {
+        return ncConfig;
+    }
+
+    public NetworkAddress getDataPort() {
+        return dataPort;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionDescriptor.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionDescriptor.java
new file mode 100644
index 0000000..9fc0916
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionDescriptor.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.control.common.job;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class PartitionDescriptor implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final PartitionId pid;
+
+    private final String nodeId;
+
+    private final TaskAttemptId producingTaskAttemptId;
+
+    private final boolean reusable;
+
+    private PartitionState state;
+
+    public PartitionDescriptor(PartitionId pid, String nodeId, TaskAttemptId producingTaskAttemptId, boolean reusable) {
+        this.pid = pid;
+        this.nodeId = nodeId;
+        this.producingTaskAttemptId = producingTaskAttemptId;
+        this.reusable = reusable;
+    }
+
+    public PartitionId getPartitionId() {
+        return pid;
+    }
+
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public TaskAttemptId getProducingTaskAttemptId() {
+        return producingTaskAttemptId;
+    }
+
+    public PartitionState getState() {
+        return state;
+    }
+
+    public void setState(PartitionState state) {
+        this.state = state;
+    }
+
+    public boolean isReusable() {
+        return reusable;
+    }
+
+    @Override
+    public String toString() {
+        return "[" + pid + ":" + nodeId + ":" + producingTaskAttemptId + (reusable ? "reusable" : "non-reusable") + " "
+                + state + "]";
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionRequest.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionRequest.java
new file mode 100644
index 0000000..ca34501
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionRequest.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.control.common.job;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class PartitionRequest implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final PartitionId pid;
+
+    private final String requestingNodeId;
+
+    private final TaskAttemptId requestingTaskAttemptId;
+
+    private final PartitionState minState;
+
+    public PartitionRequest(PartitionId pid, String requestingNodeId, TaskAttemptId requestingTaskAttemptId,
+            PartitionState minState) {
+        this.pid = pid;
+        this.requestingNodeId = requestingNodeId;
+        this.requestingTaskAttemptId = requestingTaskAttemptId;
+        this.minState = minState;
+    }
+
+    public PartitionId getPartitionId() {
+        return pid;
+    }
+
+    public String getNodeId() {
+        return requestingNodeId;
+    }
+
+    public TaskAttemptId getRequestingTaskAttemptId() {
+        return requestingTaskAttemptId;
+    }
+
+    public PartitionState getMinimumState() {
+        return minState;
+    }
+
+    @Override
+    public String toString() {
+        return "[" + pid + ":" + requestingNodeId + ":" + requestingTaskAttemptId + ":" + minState + "]";
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionState.java
similarity index 64%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
copy to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionState.java
index 88efc1f..edde0b8 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionState.java
@@ -12,20 +12,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.control.common.job;
 
-import java.io.Serializable;
+public enum PartitionState {
+    STARTED,
+    COMMITTED;
 
-public class NodeCapability implements Serializable {
-    private static final long serialVersionUID = 1L;
+    public boolean isAtLeast(PartitionState minState) {
+        switch (this) {
+            case COMMITTED:
+                return true;
 
-    private int cpuCount;
-
-    public int getCPUCount() {
-        return cpuCount;
-    }
-
-    public void setCPUCount(int cpuCount) {
-        this.cpuCount = cpuCount;
+            case STARTED:
+                return minState == STARTED;
+        }
+        return false;
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/TaskAttemptDescriptor.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/TaskAttemptDescriptor.java
new file mode 100644
index 0000000..15ba75c
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/TaskAttemptDescriptor.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.control.common.job;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+
+public class TaskAttemptDescriptor implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final TaskAttemptId taId;
+
+    private final int nPartitions;
+
+    private final int[] nInputPartitions;
+
+    private final int[] nOutputPartitions;
+
+    public TaskAttemptDescriptor(TaskAttemptId taId, int nPartitions, int[] nInputPartitions, int[] nOutputPartitions) {
+        this.taId = taId;
+        this.nPartitions = nPartitions;
+        this.nInputPartitions = nInputPartitions;
+        this.nOutputPartitions = nOutputPartitions;
+    }
+
+    public TaskAttemptId getTaskAttemptId() {
+        return taId;
+    }
+
+    public int getPartitionCount() {
+        return nPartitions;
+    }
+
+    public int[] getInputPartitionCounts() {
+        return nInputPartitions;
+    }
+
+    public int[] getOutputPartitionCounts() {
+        return nOutputPartitions;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/AbstractProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/AbstractProfile.java
similarity index 95%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/AbstractProfile.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/AbstractProfile.java
index 6ef57a6..74d6000 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/AbstractProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/AbstractProfile.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.job.profiling.om;
+package edu.uci.ics.hyracks.control.common.job.profiling.om;
 
 import java.io.Serializable;
 import java.util.HashMap;
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
similarity index 78%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobProfile.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
index 53ad0ad..5457ac5 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
@@ -1,35 +1,29 @@
-package edu.uci.ics.hyracks.api.job.profiling.om;
+package edu.uci.ics.hyracks.control.common.job.profiling.om;
 
 import java.util.HashMap;
 import java.util.Map;
-import java.util.UUID;
 
 import org.json.JSONException;
 import org.json.JSONObject;
 
+import edu.uci.ics.hyracks.api.job.JobId;
+
 public class JobProfile extends AbstractProfile {
     private static final long serialVersionUID = 1L;
 
-    private final UUID jobId;
-
-    private final int attempt;
+    private final JobId jobId;
 
     private final Map<String, JobletProfile> jobletProfiles;
 
-    public JobProfile(UUID jobId, int attempt) {
+    public JobProfile(JobId jobId) {
         this.jobId = jobId;
-        this.attempt = attempt;
         jobletProfiles = new HashMap<String, JobletProfile>();
     }
 
-    public UUID getJobId() {
+    public JobId getJobId() {
         return jobId;
     }
 
-    public int getAttempt() {
-        return attempt;
-    }
-
     public Map<String, JobletProfile> getJobletProfiles() {
         return jobletProfiles;
     }
@@ -40,7 +34,6 @@
 
         json.put("type", "job-profile");
         json.put("job-id", jobId.toString());
-        json.put("attempt", attempt);
         populateCounters(json);
         for (JobletProfile p : jobletProfiles.values()) {
             json.accumulate("joblets", p.toJSON());
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobletProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
similarity index 67%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobletProfile.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
index 66a203c..5069fc3 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobletProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
@@ -12,33 +12,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.job.profiling.om;
+package edu.uci.ics.hyracks.control.common.job.profiling.om;
 
 import java.util.HashMap;
 import java.util.Map;
-import java.util.UUID;
 
 import org.json.JSONException;
 import org.json.JSONObject;
 
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+
 public class JobletProfile extends AbstractProfile {
     private static final long serialVersionUID = 1L;
 
     private final String nodeId;
 
-    private final Map<UUID, StageletProfile> stageletProfiles;
+    private final Map<TaskAttemptId, TaskProfile> taskProfiles;
 
     public JobletProfile(String nodeId) {
         this.nodeId = nodeId;
-        stageletProfiles = new HashMap<UUID, StageletProfile>();
+        taskProfiles = new HashMap<TaskAttemptId, TaskProfile>();
     }
 
     public String getNodeId() {
         return nodeId;
     }
 
-    public Map<UUID, StageletProfile> getStageletProfiles() {
-        return stageletProfiles;
+    public Map<TaskAttemptId, TaskProfile> getTaskProfiles() {
+        return taskProfiles;
     }
 
     @Override
@@ -48,8 +49,8 @@
         json.put("type", "joblet-profile");
         json.put("node-id", nodeId.toString());
         populateCounters(json);
-        for (StageletProfile p : stageletProfiles.values()) {
-            json.accumulate("stagelets", p.toJSON());
+        for (TaskProfile p : taskProfiles.values()) {
+            json.accumulate("tasks", p.toJSON());
         }
 
         return json;
@@ -57,11 +58,11 @@
 
     public void merge(JobletProfile jp) {
         super.merge(this);
-        for (StageletProfile sp : jp.stageletProfiles.values()) {
-            if (stageletProfiles.containsKey(sp.getStageId())) {
-                stageletProfiles.get(sp.getStageId()).merge(sp);
+        for (TaskProfile tp : jp.taskProfiles.values()) {
+            if (taskProfiles.containsKey(tp.getTaskId())) {
+                taskProfiles.get(tp.getTaskId()).merge(tp);
             } else {
-                stageletProfiles.put(sp.getStageId(), sp);
+                taskProfiles.put(tp.getTaskId(), tp);
             }
         }
     }
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
new file mode 100644
index 0000000..3ab1090
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.common.job.profiling.om;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+
+public class TaskProfile extends AbstractProfile {
+    private static final long serialVersionUID = 1L;
+
+    private final TaskAttemptId taskAttemptId;
+
+    public TaskProfile(TaskAttemptId taskAttemptId) {
+        this.taskAttemptId = taskAttemptId;
+    }
+
+    public TaskAttemptId getTaskId() {
+        return taskAttemptId;
+    }
+
+    @Override
+    public JSONObject toJSON() throws JSONException {
+        JSONObject json = new JSONObject();
+
+        json.put("type", "task-profile");
+        json.put("activity-id", taskAttemptId.getTaskId().getActivityId().toString());
+        json.put("partition", taskAttemptId.getTaskId().getPartition());
+        json.put("attempt", taskAttemptId.getAttempt());
+        populateCounters(json);
+
+        return json;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index 413243e..3026359 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -15,29 +15,37 @@
 package edu.uci.ics.hyracks.control.nc;
 
 import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
-import java.util.Hashtable;
 import java.util.Map;
-import java.util.UUID;
 import java.util.concurrent.Executor;
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
 import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.job.profiling.om.JobletProfile;
-import edu.uci.ics.hyracks.api.job.profiling.om.StageletProfile;
+import edu.uci.ics.hyracks.api.naming.MultipartName;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
 import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
 import edu.uci.ics.hyracks.control.common.job.profiling.counters.Counter;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobletProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
 import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
 
 public class Joblet implements IHyracksJobletContext, ICounterContext {
@@ -47,54 +55,76 @@
 
     private final INCApplicationContext appCtx;
 
-    private final UUID jobId;
+    private final JobId jobId;
 
-    private final int attempt;
-
-    private final Map<UUID, Stagelet> stageletMap;
+    private final Map<PartitionId, IPartitionCollector> partitionRequestMap;
 
     private final Map<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>> envMap;
 
+    private final Map<TaskAttemptId, Task> taskMap;
+
     private final Map<String, Counter> counterMap;
 
+    private final Map<MultipartName, Object> localVariableMap;
+
     private final DefaultDeallocatableRegistry deallocatableRegistry;
 
     private final IWorkspaceFileFactory fileFactory;
 
-    public Joblet(NodeControllerService nodeController, UUID jobId, int attempt, INCApplicationContext appCtx) {
+    public Joblet(NodeControllerService nodeController, JobId jobId, INCApplicationContext appCtx) {
         this.nodeController = nodeController;
         this.appCtx = appCtx;
         this.jobId = jobId;
-        this.attempt = attempt;
-        stageletMap = new Hashtable<UUID, Stagelet>();
-        envMap = new Hashtable<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>>();
-        counterMap = new Hashtable<String, Counter>();
+        partitionRequestMap = new HashMap<PartitionId, IPartitionCollector>();
+        envMap = new HashMap<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>>();
+        taskMap = new HashMap<TaskAttemptId, Task>();
+        counterMap = new HashMap<String, Counter>();
+        localVariableMap = new HashMap<MultipartName, Object>();
         deallocatableRegistry = new DefaultDeallocatableRegistry();
-        fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) appCtx.getRootContext().getIOManager());
+        fileFactory = new WorkspaceFileFactory(this, (IOManager) appCtx.getRootContext().getIOManager());
     }
 
     @Override
-    public UUID getJobId() {
+    public JobId getJobId() {
         return jobId;
     }
 
-    public IOperatorEnvironment getEnvironment(IOperatorDescriptor hod, int partition) {
-        synchronized (envMap) {
-            if (!envMap.containsKey(hod.getOperatorId())) {
-                envMap.put(hod.getOperatorId(), new HashMap<Integer, IOperatorEnvironment>());
-            }
+    public synchronized IOperatorEnvironment getEnvironment(OperatorDescriptorId opId, int partition) {
+        if (!envMap.containsKey(opId)) {
+            envMap.put(opId, new HashMap<Integer, IOperatorEnvironment>());
         }
-        Map<Integer, IOperatorEnvironment> opEnvMap = envMap.get(hod.getOperatorId());
+        Map<Integer, IOperatorEnvironment> opEnvMap = envMap.get(opId);
         if (!opEnvMap.containsKey(partition)) {
-            opEnvMap.put(partition, new OperatorEnvironmentImpl());
+            opEnvMap.put(partition, new OperatorEnvironmentImpl(nodeController.getId()));
         }
         return opEnvMap.get(partition);
     }
 
+    public void addTask(Task task) {
+        taskMap.put(task.getTaskAttemptId(), task);
+    }
+
+    public Map<TaskAttemptId, Task> getTaskMap() {
+        return taskMap;
+    }
+
+    public synchronized Object lookupLocalVariable(MultipartName name) throws HyracksDataException {
+        if (!localVariableMap.containsKey(name)) {
+            throw new HyracksDataException("Unknown variable: " + name);
+        }
+        return localVariableMap.get(name);
+    }
+
+    public synchronized void setLocalVariable(MultipartName name, Object value) {
+        localVariableMap.put(name, value);
+    }
+
     private static final class OperatorEnvironmentImpl implements IOperatorEnvironment {
+        private final String nodeId;
         private final Map<String, Object> map;
 
-        public OperatorEnvironmentImpl() {
+        public OperatorEnvironmentImpl(String nodeId) {
+            this.nodeId = nodeId;
             map = new HashMap<String, Object>();
         }
 
@@ -107,47 +137,41 @@
         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 String toString() {
+            return super.toString() + "@" + nodeId;
+        }
     }
 
     public Executor getExecutor() {
         return nodeController.getExecutor();
     }
 
-    public void notifyStageletComplete(UUID stageId, int attempt, StageletProfile stats) throws Exception {
-        stageletMap.remove(stageId);
-        nodeController.notifyStageComplete(jobId, stageId, attempt, stats);
+    public synchronized void notifyTaskComplete(Task task) throws Exception {
+        taskMap.remove(task);
+        TaskProfile taskProfile = new TaskProfile(task.getTaskAttemptId());
+        task.dumpProfile(taskProfile);
+        nodeController.notifyTaskComplete(jobId, task.getTaskAttemptId(), taskProfile);
     }
 
-    public void notifyStageletFailed(UUID stageId, int attempt) throws Exception {
-        stageletMap.remove(stageId);
-        nodeController.notifyStageFailed(jobId, stageId, attempt);
+    public synchronized void notifyTaskFailed(Task task, Exception exception) {
+        taskMap.remove(task);
+        nodeController.notifyTaskFailed(jobId, task.getTaskAttemptId(), exception);
     }
 
     public NodeControllerService getNodeController() {
         return nodeController;
     }
 
-    public void dumpProfile(JobletProfile jProfile) {
-        synchronized (counterMap) {
-            Map<String, Long> counters = jProfile.getCounters();
-            for (Map.Entry<String, Counter> e : counterMap.entrySet()) {
-                counters.put(e.getKey(), e.getValue().get());
-            }
+    public synchronized void dumpProfile(JobletProfile jProfile) {
+        Map<String, Long> counters = jProfile.getCounters();
+        for (Map.Entry<String, Counter> e : counterMap.entrySet()) {
+            counters.put(e.getKey(), e.getValue().get());
         }
-        synchronized (stageletMap) {
-            for (Stagelet si : stageletMap.values()) {
-                StageletProfile sProfile = new StageletProfile(si.getStageId());
-                si.dumpProfile(sProfile);
-                jProfile.getStageletProfiles().put(si.getStageId(), sProfile);
-            }
+        for (Task task : taskMap.values()) {
+            TaskProfile taskProfile = new TaskProfile(task.getTaskAttemptId());
+            task.dumpProfile(taskProfile);
+            jProfile.getTaskProfiles().put(task.getTaskAttemptId(), taskProfile);
         }
     }
 
@@ -157,11 +181,6 @@
     }
 
     @Override
-    public int getAttempt() {
-        return attempt;
-    }
-
-    @Override
     public ICounterContext getCounterContext() {
         return this;
     }
@@ -191,23 +210,38 @@
     }
 
     @Override
-    public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
-        return fileFactory.createWorkspaceFile(prefix);
-    }
-
-    public Map<UUID, Stagelet> getStageletMap() {
-        return stageletMap;
+    public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
+        return fileFactory.createManagedWorkspaceFile(prefix);
     }
 
     @Override
-    public ICounter getCounter(String name, boolean create) {
-        synchronized (counterMap) {
-            Counter counter = counterMap.get(name);
-            if (counter == null && create) {
-                counter = new Counter(name);
-                counterMap.put(name, counter);
-            }
-            return counter;
+    public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException {
+        return fileFactory.createUnmanagedWorkspaceFile(prefix);
+    }
+
+    @Override
+    public synchronized ICounter getCounter(String name, boolean create) {
+        Counter counter = counterMap.get(name);
+        if (counter == null && create) {
+            counter = new Counter(name);
+            counterMap.put(name, counter);
+        }
+        return counter;
+    }
+
+    public synchronized void advertisePartitionRequest(TaskAttemptId taId, Collection<PartitionId> pids,
+            IPartitionCollector collector, PartitionState minState) throws Exception {
+        for (PartitionId pid : pids) {
+            partitionRequestMap.put(pid, collector);
+            PartitionRequest req = new PartitionRequest(pid, nodeController.getId(), taId, minState);
+            nodeController.getClusterController().registerPartitionRequest(req);
+        }
+    }
+
+    public synchronized void reportPartitionAvailability(PartitionChannel channel) throws HyracksException {
+        IPartitionCollector collector = partitionRequestMap.get(channel.getPartitionId());
+        if (collector != null) {
+            collector.addPartitions(Collections.singleton(channel));
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java
index d8c978a..dde7abc 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java
@@ -18,7 +18,7 @@
 
 import edu.uci.ics.dcache.client.DCacheClient;
 import edu.uci.ics.dcache.client.DCacheClientConfig;
-import edu.uci.ics.hyracks.api.control.NCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 
 public class NCDriver {
     public static void main(String args[]) throws Exception {
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index be48ae6..4eb1a03 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -19,7 +19,7 @@
 import java.io.OutputStream;
 import java.io.Serializable;
 import java.net.InetAddress;
-import java.nio.ByteBuffer;
+import java.net.InetSocketAddress;
 import java.rmi.registry.LocateRegistry;
 import java.rmi.registry.Registry;
 import java.text.MessageFormat;
@@ -28,11 +28,9 @@
 import java.util.Hashtable;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.StringTokenizer;
 import java.util.Timer;
 import java.util.TimerTask;
-import java.util.UUID;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Executors;
 import java.util.logging.Level;
@@ -47,47 +45,51 @@
 import org.apache.http.impl.client.DefaultHttpClient;
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
-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.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.control.IClusterController;
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.api.control.NCConfig;
-import edu.uci.ics.hyracks.api.control.NodeCapability;
-import edu.uci.ics.hyracks.api.control.NodeParameters;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.Direction;
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorInstanceId;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.IODeviceHandle;
-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.profiling.counters.ICounter;
-import edu.uci.ics.hyracks.api.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.api.job.profiling.om.JobletProfile;
-import edu.uci.ics.hyracks.api.job.profiling.om.StageletProfile;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.naming.MultipartName;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
 import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.common.base.IClusterController;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NodeCapability;
+import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
+import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobletProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
 import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
-import edu.uci.ics.hyracks.control.nc.comm.ConnectionManager;
-import edu.uci.ics.hyracks.control.nc.comm.DemuxDataReceiveListenerFactory;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.control.nc.runtime.OperatorRunnable;
+import edu.uci.ics.hyracks.control.nc.net.ConnectionManager;
+import edu.uci.ics.hyracks.control.nc.net.NetworkInputChannel;
+import edu.uci.ics.hyracks.control.nc.partitions.MaterializedPartitionWriter;
+import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
+import edu.uci.ics.hyracks.control.nc.partitions.PipelinedPartition;
+import edu.uci.ics.hyracks.control.nc.partitions.ReceiveSideMaterializingCollector;
 import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
 
 public class NodeControllerService extends AbstractRemoteService implements INodeController {
@@ -103,13 +105,15 @@
 
     private final NodeCapability nodeCapability;
 
+    private final PartitionManager partitionManager;
+
     private final ConnectionManager connectionManager;
 
     private final Timer timer;
 
     private IClusterController ccs;
 
-    private final Map<UUID, Joblet> jobletMap;
+    private final Map<JobId, Joblet> jobletMap;
 
     private final Executor executor;
 
@@ -129,13 +133,20 @@
         }
         nodeCapability = computeNodeCapability();
         connectionManager = new ConnectionManager(ctx, getIpAddress(ncConfig));
-        jobletMap = new Hashtable<UUID, Joblet>();
+        partitionManager = new PartitionManager(this);
+        connectionManager.setPartitionRequestListener(partitionManager);
+
+        jobletMap = new Hashtable<JobId, Joblet>();
         timer = new Timer(true);
         serverCtx = new ServerContext(ServerContext.ServerType.NODE_CONTROLLER, new File(new File(
                 NodeControllerService.class.getName()), id));
         applications = new Hashtable<String, NCApplicationContext>();
     }
 
+    public IHyracksRootContext getRootContext() {
+        return ctx;
+    }
+
     private static List<IODeviceHandle> getDevices(String ioDevices) {
         List<IODeviceHandle> devices = new ArrayList<IODeviceHandle>();
         StringTokenizer tok = new StringTokenizer(ioDevices, ",");
@@ -152,7 +163,8 @@
         connectionManager.start();
         Registry registry = LocateRegistry.getRegistry(ncConfig.ccHost, ncConfig.ccPort);
         IClusterController cc = (IClusterController) registry.lookup(IClusterController.class.getName());
-        this.nodeParameters = cc.registerNode(this);
+        this.nodeParameters = cc.registerNode(new NodeRegistration(this, id, ncConfig, connectionManager
+                .getNetworkAddress()));
 
         // Schedule heartbeat generator.
         timer.schedule(new HeartbeatTask(cc), 0, nodeParameters.getHeartbeatPeriod());
@@ -168,6 +180,7 @@
     @Override
     public void stop() throws Exception {
         LOGGER.log(Level.INFO, "Stopping NodeControllerService");
+        partitionManager.close();
         connectionManager.stop();
         LOGGER.log(Level.INFO, "Stopped NodeControllerService");
     }
@@ -186,6 +199,14 @@
         return connectionManager;
     }
 
+    public PartitionManager getPartitionManager() {
+        return partitionManager;
+    }
+
+    public IClusterController getClusterController() {
+        return ccs;
+    }
+
     private static NodeCapability computeNodeCapability() {
         NodeCapability nc = new NodeCapability();
         nc.setCPUCount(Runtime.getRuntime().availableProcessors());
@@ -210,14 +231,13 @@
     }
 
     @Override
-    public Map<PortInstanceId, Endpoint> initializeJobletPhase1(String appName, UUID jobId, int attempt,
-            byte[] planBytes, UUID stageId, Map<ActivityNodeId, Set<Integer>> tasks,
-            Map<OperatorDescriptorId, Integer> opNumPartitions) throws Exception {
+    public void startTasks(String appName, final JobId jobId, byte[] jagBytes,
+            List<TaskAttemptDescriptor> taskDescriptors,
+            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap, byte[] ctxVarBytes) throws Exception {
         try {
-            LOGGER.log(Level.INFO, String.valueOf(jobId) + "[" + id + ":" + stageId + "]: Initializing Joblet Phase 1");
-
             NCApplicationContext appCtx = applications.get(appName);
-            final JobPlan plan = (JobPlan) appCtx.deserialize(planBytes);
+            final JobActivityGraph plan = (JobActivityGraph) appCtx.deserialize(jagBytes);
+            Map<MultipartName, Object> ctxVarMap = (Map<MultipartName, Object>) appCtx.deserialize(ctxVarBytes);
 
             IRecordDescriptorProvider rdp = new IRecordDescriptorProvider() {
                 @Override
@@ -231,156 +251,62 @@
                 }
             };
 
-            final Joblet joblet = getOrCreateLocalJoblet(jobId, attempt, appCtx);
+            final Joblet joblet = getOrCreateLocalJoblet(jobId, appCtx);
 
-            Stagelet stagelet = new Stagelet(joblet, stageId, attempt, id);
-            joblet.setStagelet(stageId, 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 : tasks.keySet()) {
-                IActivityNode han = plan.getActivityNodeMap().get(hanId);
-                if (LOGGER.isLoggable(Level.FINEST)) {
-                    LOGGER.finest("Initializing " + hanId + " -> " + han);
+            for (TaskAttemptDescriptor td : taskDescriptors) {
+                TaskAttemptId taId = td.getTaskAttemptId();
+                TaskId tid = taId.getTaskId();
+                IActivity han = plan.getActivityNodeMap().get(tid.getActivityId());
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Initializing " + taId + " -> " + han);
                 }
-                IOperatorDescriptor op = han.getOwner();
-                List<IConnectorDescriptor> inputs = plan.getTaskInputs(hanId);
-                for (int i : tasks.get(hanId)) {
-                    IOperatorNodePushable hon = han.createPushRuntime(stagelet, joblet.getEnvironment(op, i), rdp, i,
-                            opNumPartitions.get(op.getOperatorId()));
-                    OperatorRunnable or = new OperatorRunnable(stagelet, hon, inputs == null ? 0 : inputs.size(),
-                            executor);
-                    stagelet.setOperator(op.getOperatorId(), i, or);
-                    if (inputs != null) {
-                        for (int j = 0; j < inputs.size(); ++j) {
-                            IConnectorDescriptor conn = inputs.get(j);
-                            OperatorDescriptorId producerOpId = plan.getJobSpecification().getProducer(conn)
-                                    .getOperatorId();
-                            OperatorDescriptorId consumerOpId = plan.getJobSpecification().getConsumer(conn)
-                                    .getOperatorId();
-                            Endpoint endpoint = new Endpoint(connectionManager.getNetworkAddress(), i);
-                            endpointList.add(endpoint);
-                            DemuxDataReceiveListenerFactory drlf = new DemuxDataReceiveListenerFactory(stagelet, jobId,
-                                    stageId);
-                            connectionManager.acceptConnection(endpoint.getEndpointId(), drlf);
-                            PortInstanceId piId = new PortInstanceId(op.getOperatorId(), 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(stagelet, conn, drlf, i, plan, stagelet,
-                                    opNumPartitions.get(producerOpId), opNumPartitions.get(consumerOpId));
-                            or.setFrameReader(j, reader);
+                final int partition = tid.getPartition();
+                Map<MultipartName, Object> inputGlobalVariables = createInputGlobalVariables(ctxVarMap, han);
+                Task task = new Task(joblet, taId, han.getClass().getName(), executor);
+                IOperatorEnvironment env = joblet.getEnvironment(tid.getActivityId().getOperatorDescriptorId(),
+                        tid.getPartition());
+                IOperatorNodePushable operator = han.createPushRuntime(task, env, rdp, partition,
+                        td.getPartitionCount());
+
+                List<IPartitionCollector> collectors = new ArrayList<IPartitionCollector>();
+
+                List<IConnectorDescriptor> inputs = plan.getActivityInputConnectorDescriptors(tid.getActivityId());
+                if (inputs != null) {
+                    for (int i = 0; i < inputs.size(); ++i) {
+                        IConnectorDescriptor conn = inputs.get(i);
+                        IConnectorPolicy cPolicy = connectorPoliciesMap.get(conn.getConnectorId());
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info("input: " + i + ": " + conn.getConnectorId());
                         }
+                        RecordDescriptor recordDesc = plan.getJobSpecification().getConnectorRecordDescriptor(conn);
+                        IPartitionCollector collector = createPartitionCollector(td, partition, task, i, conn,
+                                recordDesc, cPolicy);
+                        collectors.add(collector);
                     }
-                    honMap.put(new OperatorInstanceId(op.getOperatorId(), i), or);
                 }
-            }
+                List<IConnectorDescriptor> outputs = plan.getActivityOutputConnectorDescriptors(tid.getActivityId());
+                if (outputs != null) {
+                    for (int i = 0; i < outputs.size(); ++i) {
+                        final IConnectorDescriptor conn = outputs.get(i);
+                        RecordDescriptor recordDesc = plan.getJobSpecification().getConnectorRecordDescriptor(conn);
+                        IConnectorPolicy cPolicy = connectorPoliciesMap.get(conn.getConnectorId());
 
-            stagelet.setEndpointList(endpointList);
+                        IPartitionWriterFactory pwFactory = createPartitionWriterFactory(cPolicy, jobId, conn,
+                                partition, taId);
 
-            return portMap;
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw e;
-        }
-    }
-
-    private IFrameReader createReader(final IHyracksStageletContext stageletContext, final IConnectorDescriptor conn,
-            IConnectionDemultiplexer demux, final int receiverIndex, JobPlan plan, final Stagelet stagelet,
-            int nProducerCount, int nConsumerCount) throws HyracksDataException {
-        final IFrameReader reader = conn.createReceiveSideReader(stageletContext, plan.getJobSpecification()
-                .getConnectorRecordDescriptor(conn), demux, receiverIndex, nProducerCount, nConsumerCount);
-
-        return plan.getJobFlags().contains(JobFlag.PROFILE_RUNTIME) ? new IFrameReader() {
-            private ICounter openCounter = stageletContext.getCounterContext().getCounter(
-                    conn.getConnectorId().getId() + ".receiver." + receiverIndex + ".open", true);
-            private ICounter closeCounter = stageletContext.getCounterContext().getCounter(
-                    conn.getConnectorId().getId() + ".receiver." + receiverIndex + ".close", true);
-            private ICounter frameCounter = stageletContext.getCounterContext().getCounter(
-                    conn.getConnectorId().getId() + ".receiver." + receiverIndex + ".nextFrame", true);
-
-            @Override
-            public void open() throws HyracksDataException {
-                reader.open();
-                openCounter.update(1);
-            }
-
-            @Override
-            public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                boolean status = reader.nextFrame(buffer);
-                if (status) {
-                    frameCounter.update(1);
-                }
-                return status;
-            }
-
-            @Override
-            public void close() throws HyracksDataException {
-                reader.close();
-                closeCounter.update(1);
-            }
-        } : reader;
-    }
-
-    @Override
-    public void initializeJobletPhase2(String appName, UUID jobId, byte[] planBytes, UUID stageId,
-            Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Integer> opNumPartitions,
-            final Map<PortInstanceId, Endpoint> globalPortMap) throws Exception {
-        try {
-            LOGGER.log(Level.INFO, String.valueOf(jobId) + "[" + id + ":" + stageId + "]: Initializing Joblet Phase 2");
-            ApplicationContext appCtx = applications.get(appName);
-            final JobPlan plan = (JobPlan) appCtx.deserialize(planBytes);
-
-            final Joblet ji = getLocalJoblet(jobId);
-            final Stagelet stagelet = (Stagelet) ji.getStagelet(stageId);
-            final Map<OperatorInstanceId, OperatorRunnable> honMap = stagelet.getOperatorMap();
-
-            final JobSpecification spec = plan.getJobSpecification();
-
-            for (ActivityNodeId hanId : tasks.keySet()) {
-                IActivityNode han = plan.getActivityNodeMap().get(hanId);
-                IOperatorDescriptor op = han.getOwner();
-                List<IConnectorDescriptor> outputs = plan.getTaskOutputs(hanId);
-                for (int i : tasks.get(hanId)) {
-                    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);
-                            OperatorDescriptorId producerOpId = plan.getJobSpecification().getProducer(conn)
-                                    .getOperatorId();
-                            OperatorDescriptorId consumerOpId = plan.getJobSpecification().getConsumer(conn)
-                                    .getOperatorId();
-                            final int senderIndex = i;
-                            IEndpointDataWriterFactory edwFactory = new IEndpointDataWriterFactory() {
-                                @Override
-                                public IFrameWriter createFrameWriter(int index) throws HyracksDataException {
-                                    PortInstanceId piId = new PortInstanceId(spec.getConsumer(conn).getOperatorId(),
-                                            Direction.INPUT, spec.getConsumerInputIndex(conn), index);
-                                    Endpoint ep = globalPortMap.get(piId);
-                                    if (ep == null) {
-                                        LOGGER.info("Got null Endpoint for " + piId);
-                                        throw new NullPointerException();
-                                    }
-                                    if (LOGGER.isLoggable(Level.FINEST)) {
-                                        LOGGER.finest("Probed endpoint " + piId + " -> " + ep);
-                                    }
-                                    return createWriter(stagelet, connectionManager.connect(ep.getNetworkAddress(),
-                                            ep.getEndpointId(), senderIndex), plan, conn, senderIndex, index, stagelet);
-                                }
-                            };
-                            or.setFrameWriter(j, conn.createSendSideWriter(stagelet, plan.getJobSpecification()
-                                    .getConnectorRecordDescriptor(conn), edwFactory, i, opNumPartitions
-                                    .get(producerOpId), opNumPartitions.get(consumerOpId)), spec
-                                    .getConnectorRecordDescriptor(conn));
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info("output: " + i + ": " + conn.getConnectorId());
                         }
+                        IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition,
+                                td.getPartitionCount(), td.getOutputPartitionCounts()[i]);
+                        operator.setOutputFrameWriter(i, writer, recordDesc);
                     }
-                    stagelet.installRunnable(new OperatorInstanceId(op.getOperatorId(), i));
                 }
+
+                task.setTaskRuntime(collectors.toArray(new IPartitionCollector[collectors.size()]), operator);
+                joblet.addTask(task);
+
+                task.start();
             }
         } catch (Exception e) {
             e.printStackTrace();
@@ -388,68 +314,55 @@
         }
     }
 
-    private IFrameWriter createWriter(final IHyracksStageletContext stageletContext, final IFrameWriter writer,
-            JobPlan plan, final IConnectorDescriptor conn, final int senderIndex, final int receiverIndex,
-            final Stagelet stagelet) throws HyracksDataException {
-        return plan.getJobFlags().contains(JobFlag.PROFILE_RUNTIME) ? new IFrameWriter() {
-            private ICounter openCounter = stageletContext.getCounterContext().getCounter(
-                    conn.getConnectorId().getId() + ".sender." + senderIndex + "." + receiverIndex + ".open", true);
-            private ICounter closeCounter = stageletContext.getCounterContext().getCounter(
-                    conn.getConnectorId().getId() + ".sender." + senderIndex + "." + receiverIndex + ".close", true);
-            private ICounter frameCounter = stageletContext.getCounterContext()
-                    .getCounter(
-                            conn.getConnectorId().getId() + ".sender." + senderIndex + "." + receiverIndex
-                                    + ".nextFrame", true);
-
-            @Override
-            public void open() throws HyracksDataException {
-                writer.open();
-                openCounter.update(1);
-            }
-
-            @Override
-            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                frameCounter.update(1);
-                writer.nextFrame(buffer);
-            }
-
-            @Override
-            public void close() throws HyracksDataException {
-                closeCounter.update(1);
-                writer.close();
-            }
-
-            @Override
-            public void flush() throws HyracksDataException {
-                writer.flush();
-            }
-        } : writer;
+    private Map<MultipartName, Object> createInputGlobalVariables(Map<MultipartName, Object> ctxVarMap, IActivity han) {
+        Map<MultipartName, Object> gVars = new HashMap<MultipartName, Object>();
+        //        for (MultipartName inVar : han.getInputVariables()) {
+        //            gVars.put(inVar, ctxVarMap.get(inVar));
+        //        }
+        return gVars;
     }
 
-    @Override
-    public void commitJobletInitialization(UUID jobId, UUID stageId) throws Exception {
-        final Joblet ji = getLocalJoblet(jobId);
-        Stagelet si = (Stagelet) ji.getStagelet(stageId);
-        for (Endpoint e : si.getEndpointList()) {
-            connectionManager.unacceptConnection(e.getEndpointId());
+    private IPartitionCollector createPartitionCollector(TaskAttemptDescriptor td, final int partition, Task task,
+            int i, IConnectorDescriptor conn, RecordDescriptor recordDesc, IConnectorPolicy cPolicy)
+            throws HyracksDataException {
+        IPartitionCollector collector = conn.createPartitionCollector(task, recordDesc, partition,
+                td.getInputPartitionCounts()[i], td.getPartitionCount());
+        if (cPolicy.materializeOnReceiveSide()) {
+            return new ReceiveSideMaterializingCollector(ctx, partitionManager, collector, task.getTaskAttemptId(),
+                    executor);
+        } else {
+            return collector;
         }
-        si.setEndpointList(null);
     }
 
-    private Joblet getLocalJoblet(UUID jobId) throws Exception {
+    private IPartitionWriterFactory createPartitionWriterFactory(IConnectorPolicy cPolicy, final JobId jobId,
+            final IConnectorDescriptor conn, final int senderIndex, final TaskAttemptId taId) {
+        if (cPolicy.materializeOnSendSide()) {
+            return new IPartitionWriterFactory() {
+                @Override
+                public IFrameWriter createFrameWriter(int receiverIndex) throws HyracksDataException {
+                    return new MaterializedPartitionWriter(ctx, partitionManager, new PartitionId(jobId,
+                            conn.getConnectorId(), senderIndex, receiverIndex), taId, executor);
+                }
+            };
+        } else {
+            return new IPartitionWriterFactory() {
+                @Override
+                public IFrameWriter createFrameWriter(int receiverIndex) throws HyracksDataException {
+                    return new PipelinedPartition(partitionManager, new PartitionId(jobId, conn.getConnectorId(),
+                            senderIndex, receiverIndex), taId);
+                }
+            };
+        }
+    }
+
+    private synchronized Joblet getOrCreateLocalJoblet(JobId jobId, INCApplicationContext appCtx) throws Exception {
         Joblet ji = jobletMap.get(jobId);
-        return ji;
-    }
-
-    private Joblet getOrCreateLocalJoblet(UUID jobId, int attempt, INCApplicationContext appCtx) throws Exception {
-        synchronized (jobletMap) {
-            Joblet ji = jobletMap.get(jobId);
-            if (ji == null || ji.getAttempt() != attempt) {
-                ji = new Joblet(this, jobId, attempt, appCtx);
-                jobletMap.put(jobId, ji);
-            }
-            return ji;
+        if (ji == null) {
+            ji = new Joblet(this, jobId, appCtx);
+            jobletMap.put(jobId, ji);
         }
+        return ji;
     }
 
     public Executor getExecutor() {
@@ -457,43 +370,31 @@
     }
 
     @Override
-    public void cleanUpJob(UUID jobId) throws Exception {
+    public void cleanUpJob(JobId jobId) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Cleaning up after job: " + jobId);
         }
         Joblet joblet = jobletMap.remove(jobId);
         if (joblet != null) {
+            partitionManager.unregisterPartitions(jobId);
             joblet.close();
         }
-        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, int attempt, StageletProfile stats) throws Exception {
+    public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, TaskProfile taskProfile) throws Exception {
         try {
-            ccs.notifyStageletComplete(jobId, stageId, attempt, id, stats);
+            ccs.notifyTaskComplete(jobId, taskId, id, taskProfile);
         } catch (Exception e) {
             e.printStackTrace();
             throw e;
         }
     }
 
-    public void notifyStageFailed(UUID jobId, UUID stageId, int attempt) throws Exception {
+    public void notifyTaskFailed(JobId jobId, TaskAttemptId taskId, Exception exception) {
         try {
-            ccs.notifyStageletFailure(jobId, stageId, attempt, id);
+            ccs.notifyTaskFailure(jobId, taskId, id, exception);
         } catch (Exception e) {
             e.printStackTrace();
-            throw e;
         }
     }
 
@@ -538,7 +439,7 @@
                 synchronized (NodeControllerService.this) {
                     profiles = new ArrayList<JobProfile>();
                     for (Joblet ji : jobletMap.values()) {
-                        profiles.add(new JobProfile(ji.getJobId(), ji.getAttempt()));
+                        profiles.add(new JobProfile(ji.getJobId()));
                     }
                 }
                 for (JobProfile jProfile : profiles) {
@@ -562,20 +463,20 @@
     }
 
     @Override
-    public void abortJoblet(UUID jobId, int attempt) throws Exception {
+    public synchronized void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Aborting Job: " + jobId + ":" + attempt);
+            LOGGER.info("Aborting Tasks: " + jobId + ":" + tasks);
         }
         Joblet ji = jobletMap.get(jobId);
         if (ji != null) {
-            if (ji.getAttempt() == attempt) {
-                jobletMap.remove(jobId);
+            Map<TaskAttemptId, Task> taskMap = ji.getTaskMap();
+            for (TaskAttemptId taId : tasks) {
+                Task task = taskMap.get(taId);
+                if (task != null) {
+                    task.abort();
+                }
             }
-            for (Stagelet stagelet : ji.getStageletMap().values()) {
-                stagelet.abort();
-                stagelet.close();
-                connectionManager.abortConnections(jobId, stagelet.getStageId());
-            }
+            ji.close();
         }
     }
 
@@ -616,4 +517,14 @@
             appCtx.deinitialize();
         }
     }
+
+    @Override
+    public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception {
+        Joblet ji = jobletMap.get(pid.getJobId());
+        if (ji != null) {
+            PartitionChannel channel = new PartitionChannel(pid, new NetworkInputChannel(ctx, connectionManager,
+                    new InetSocketAddress(networkAddress.getIpAddress(), networkAddress.getPort()), pid, 1));
+            ji.reportPartitionAvailability(channel);
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
deleted file mode 100644
index 68b2cad..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
+++ /dev/null
@@ -1,242 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.nc;
-
-import java.nio.ByteBuffer;
-import java.rmi.RemoteException;
-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.context.IHyracksJobletContext;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorInstanceId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
-import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
-import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.job.profiling.om.StageletProfile;
-import edu.uci.ics.hyracks.api.resources.IDeallocatable;
-import edu.uci.ics.hyracks.control.common.job.profiling.counters.Counter;
-import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
-import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
-import edu.uci.ics.hyracks.control.nc.runtime.OperatorRunnable;
-
-public class Stagelet implements IHyracksStageletContext, ICounterContext {
-    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 int attempt;
-
-    private final Map<OperatorInstanceId, OperatorRunnable> honMap;
-
-    private final Map<String, Counter> counterMap;
-
-    private final IWorkspaceFileFactory fileFactory;
-
-    private List<Endpoint> endpointList;
-
-    private boolean started;
-
-    private volatile boolean abort;
-
-    private final Set<OperatorInstanceId> pendingOperators;
-
-    private final DefaultDeallocatableRegistry deallocatableRegistry;
-
-    public Stagelet(Joblet joblet, UUID stageId, int attempt, String nodeId) throws RemoteException {
-        this.joblet = joblet;
-        this.stageId = stageId;
-        this.attempt = attempt;
-        pendingOperators = new HashSet<OperatorInstanceId>();
-        started = false;
-        honMap = new HashMap<OperatorInstanceId, OperatorRunnable>();
-        counterMap = new HashMap<String, Counter>();
-        deallocatableRegistry = new DefaultDeallocatableRegistry();
-        fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) joblet.getIOManager());
-    }
-
-    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;
-        notifyAll();
-    }
-
-    public synchronized void abort() {
-        this.abort = true;
-        for (OperatorRunnable r : honMap.values()) {
-            r.abort();
-        }
-    }
-
-    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;
-                }
-                if (abort) {
-                    return;
-                }
-                try {
-                    LOGGER.log(Level.INFO, joblet.getJobId() + ":" + stageId + ":" + opIId.getOperatorId() + ":"
-                            + opIId.getPartition() + "(" + hon + ")" + ": STARTED");
-                    hon.run();
-                    LOGGER.log(Level.INFO, joblet.getJobId() + ":" + stageId + ":" + opIId.getOperatorId() + ":"
-                            + opIId.getPartition() + "(" + hon + ")" + ": FINISHED");
-                    notifyOperatorCompletion(opIId);
-                } catch (Exception e) {
-                    LOGGER.log(Level.INFO, joblet.getJobId() + ":" + stageId + ":" + opIId.getOperatorId() + ":"
-                            + opIId.getPartition() + "(" + hon + ")" + ": ABORTED");
-                    e.printStackTrace();
-                    notifyOperatorFailure(opIId);
-                }
-            }
-        });
-    }
-
-    protected void notifyOperatorCompletion(OperatorInstanceId opIId) {
-        boolean done = false;
-        synchronized (pendingOperators) {
-            pendingOperators.remove(opIId);
-            done = pendingOperators.isEmpty();
-        }
-        if (done) {
-            try {
-                StageletProfile sProfile = new StageletProfile(stageId);
-                dumpProfile(sProfile);
-                close();
-                joblet.notifyStageletComplete(stageId, attempt, sProfile);
-            } catch (Exception e) {
-                e.printStackTrace();
-            }
-        }
-    }
-
-    protected synchronized void notifyOperatorFailure(OperatorInstanceId opIId) {
-        abort();
-        try {
-            joblet.notifyStageletFailed(stageId, attempt);
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-
-    private synchronized void waitUntilStarted() throws InterruptedException {
-        while (!started && !abort) {
-            wait();
-        }
-    }
-
-    public synchronized void dumpProfile(StageletProfile sProfile) {
-        Map<String, Long> dumpMap = sProfile.getCounters();
-        for (Counter c : counterMap.values()) {
-            dumpMap.put(c.getName(), c.get());
-        }
-    }
-
-    @Override
-    public IHyracksJobletContext getJobletContext() {
-        return joblet;
-    }
-
-    @Override
-    public UUID getStageId() {
-        return stageId;
-    }
-
-    @Override
-    public ICounterContext getCounterContext() {
-        return this;
-    }
-
-    @Override
-    public void registerDeallocatable(IDeallocatable deallocatable) {
-        deallocatableRegistry.registerDeallocatable(deallocatable);
-    }
-
-    public void close() {
-        deallocatableRegistry.close();
-    }
-
-    @Override
-    public ByteBuffer allocateFrame() {
-        return joblet.allocateFrame();
-    }
-
-    @Override
-    public int getFrameSize() {
-        return joblet.getFrameSize();
-    }
-
-    @Override
-    public IIOManager getIOManager() {
-        return joblet.getIOManager();
-    }
-
-    @Override
-    public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
-        return fileFactory.createWorkspaceFile(prefix);
-    }
-
-    @Override
-    public synchronized ICounter getCounter(String name, boolean create) {
-        Counter counter = counterMap.get(name);
-        if (counter == null && create) {
-            counter = new Counter(name);
-            counterMap.put(name, counter);
-        }
-        return counter;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
new file mode 100644
index 0000000..e65f59e
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -0,0 +1,298 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Semaphore;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
+import edu.uci.ics.hyracks.api.naming.MultipartName;
+import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.api.workflow.variables.WorkflowVariableDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
+import edu.uci.ics.hyracks.control.common.job.profiling.counters.Counter;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
+
+public class Task implements IHyracksTaskContext, ICounterContext, Runnable {
+    private final Joblet joblet;
+
+    private final TaskAttemptId taskAttemptId;
+
+    private final String displayName;
+
+    private final Executor executor;
+
+    private final IWorkspaceFileFactory fileFactory;
+
+    private final DefaultDeallocatableRegistry deallocatableRegistry;
+
+    private final Map<String, Counter> counterMap;
+
+    private final Map<MultipartName, Object> inputGlobalVariables;
+
+    private final Map<MultipartName, Object> outputVariables;
+
+    private final Map<MultipartName, WorkflowVariableDescriptor> outputVariableDescriptorMap;
+
+    private IPartitionCollector[] collectors;
+
+    private IOperatorNodePushable operator;
+
+    private volatile boolean aborted;
+
+    public Task(Joblet joblet, TaskAttemptId taskId, String displayName,
+            Executor executor) {
+        this.joblet = joblet;
+        this.taskAttemptId = taskId;
+        this.displayName = displayName;
+        this.executor = executor;
+        fileFactory = new WorkspaceFileFactory(this, (IOManager) joblet.getIOManager());
+        deallocatableRegistry = new DefaultDeallocatableRegistry();
+        counterMap = new HashMap<String, Counter>();
+//        this.inputGlobalVariables = inputGlobalVariables;
+        inputGlobalVariables = Collections.emptyMap();
+        outputVariables = new HashMap<MultipartName, Object>();
+        outputVariableDescriptorMap = new HashMap<MultipartName, WorkflowVariableDescriptor>();
+//        for (WorkflowVariableDescriptor wvd : outputVariableDescriptors) {
+//            outputVariableDescriptorMap.put(wvd.getName(), wvd);
+//        }
+    }
+
+    public void setTaskRuntime(IPartitionCollector[] collectors, IOperatorNodePushable operator) {
+        this.collectors = collectors;
+        this.operator = operator;
+    }
+
+    @Override
+    public ByteBuffer allocateFrame() {
+        return joblet.allocateFrame();
+    }
+
+    @Override
+    public int getFrameSize() {
+        return joblet.getFrameSize();
+    }
+
+    @Override
+    public IIOManager getIOManager() {
+        return joblet.getIOManager();
+    }
+
+    @Override
+    public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException {
+        return fileFactory.createUnmanagedWorkspaceFile(prefix);
+    }
+
+    @Override
+    public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
+        return fileFactory.createManagedWorkspaceFile(prefix);
+    }
+
+    @Override
+    public void registerDeallocatable(IDeallocatable deallocatable) {
+        deallocatableRegistry.registerDeallocatable(deallocatable);
+    }
+
+    @Override
+    public Object lookupGlobalVariable(ActivityId producerActivity, int partition, String varName)
+            throws HyracksDataException {
+        MultipartName var = new MultipartName(producerActivity, partition, varName);
+        if (!inputGlobalVariables.containsKey(var)) {
+            throw new HyracksDataException("Unknown Variable: " + var);
+        }
+        return inputGlobalVariables.get(var);
+    }
+
+    @Override
+    public Object lookupLocalVariable(ActivityId producerActivity, int partition, String varName)
+            throws HyracksDataException {
+        return joblet.lookupLocalVariable(new MultipartName(producerActivity, partition, varName));
+    }
+
+    @Override
+    public void setVariable(String name, Object value) {
+        outputVariables.put(new MultipartName(taskAttemptId.getTaskId().getActivityId(), taskAttemptId.getTaskId()
+                .getPartition(), name), value);
+    }
+
+    public void close() {
+        deallocatableRegistry.close();
+    }
+
+    @Override
+    public IHyracksJobletContext getJobletContext() {
+        return joblet;
+    }
+
+    @Override
+    public TaskAttemptId getTaskAttemptId() {
+        return taskAttemptId;
+    }
+
+    @Override
+    public ICounter getCounter(String name, boolean create) {
+        Counter counter = counterMap.get(name);
+        if (counter == null && create) {
+            counter = new Counter(name);
+            counterMap.put(name, counter);
+        }
+        return counter;
+    }
+
+    @Override
+    public ICounterContext getCounterContext() {
+        return this;
+    }
+
+    public synchronized void dumpProfile(TaskProfile tProfile) {
+        Map<String, Long> dumpMap = tProfile.getCounters();
+        for (Counter c : counterMap.values()) {
+            dumpMap.put(c.getName(), c.get());
+        }
+    }
+
+    public void start() throws HyracksException {
+        aborted = false;
+        joblet.getExecutor().execute(this);
+    }
+
+    public void abort() {
+        aborted = true;
+        for (IPartitionCollector c : collectors) {
+            c.abort();
+        }
+    }
+
+    @Override
+    public void run() {
+        Thread ct = Thread.currentThread();
+        String threadName = ct.getName();
+        try {
+            ct.setName(displayName + ": " + taskAttemptId);
+            operator.initialize();
+            try {
+                if (collectors.length > 0) {
+                    final Semaphore sem = new Semaphore(collectors.length - 1);
+                    for (int i = 1; i < collectors.length; ++i) {
+                        final IPartitionCollector collector = collectors[i];
+                        final IFrameWriter writer = operator.getInputFrameWriter(i);
+                        sem.acquire();
+                        executor.execute(new Runnable() {
+                            public void run() {
+                                try {
+                                    pushFrames(collector, writer);
+                                } catch (HyracksDataException e) {
+                                } finally {
+                                    sem.release();
+                                }
+                            }
+                        });
+                    }
+                    try {
+                        pushFrames(collectors[0], operator.getInputFrameWriter(0));
+                    } finally {
+                        sem.acquire(collectors.length - 1);
+                    }
+                }
+            } finally {
+                operator.deinitialize();
+            }
+            Map<MultipartName, Object> outputGlobalVariables = new HashMap<MultipartName, Object>();
+            for (Map.Entry<MultipartName, Object> e : outputVariables.entrySet()) {
+                MultipartName varName = e.getKey();
+                WorkflowVariableDescriptor wvd = outputVariableDescriptorMap.get(varName);
+                if (wvd == null) {
+                    throw new HyracksDataException("Unknown variable found: " + varName);
+                }
+                switch (wvd.getScope()) {
+                    case LOCAL:
+                        joblet.setLocalVariable(varName, e.getValue());
+                        break;
+
+                    case GLOBAL:
+                        outputGlobalVariables.put(varName, e.getValue());
+                        break;
+                }
+            }
+            joblet.notifyTaskComplete(this);
+        } catch (Exception e) {
+            e.printStackTrace();
+            joblet.notifyTaskFailed(this, e);
+        } finally {
+            ct.setName(threadName);
+            close();
+        }
+    }
+
+    private void pushFrames(IPartitionCollector collector, IFrameWriter writer) throws HyracksDataException {
+        if (aborted) {
+            return;
+        }
+        try {
+            collector.open();
+            try {
+                joblet.advertisePartitionRequest(taskAttemptId, collector.getRequiredPartitionIds(), collector,
+                        PartitionState.STARTED);
+                IFrameReader reader = collector.getReader();
+                reader.open();
+                try {
+                    writer.open();
+                    try {
+                        ByteBuffer buffer = allocateFrame();
+                        while (reader.nextFrame(buffer)) {
+                            if (aborted) {
+                                return;
+                            }
+                            buffer.flip();
+                            writer.nextFrame(buffer);
+                            buffer.compact();
+                        }
+                    } finally {
+                        writer.close();
+                    }
+                } finally {
+                    reader.close();
+                }
+            } finally {
+                collector.close();
+            }
+        } catch (HyracksException e) {
+            throw new HyracksDataException(e);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionEntry.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionEntry.java
deleted file mode 100644
index e49f435..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionEntry.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.nc.comm;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
-import java.util.UUID;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
-import edu.uci.ics.hyracks.api.comm.IDataReceiveListener;
-import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-
-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;
-
-    private UUID jobId;
-
-    private UUID stageId;
-
-    private boolean aborted;
-
-    public ConnectionEntry(IHyracksRootContext ctx, SocketChannel socketChannel, SelectionKey key) {
-        this.socketChannel = socketChannel;
-        readBuffer = ctx.allocateFrame();
-        readBuffer.clear();
-        writeBuffer = ctx.allocateFrame();
-        writeBuffer.clear();
-        this.key = key;
-    }
-
-    public SocketChannel getSocketChannel() {
-        return socketChannel;
-    }
-
-    public boolean dispatch(SelectionKey key) throws IOException {
-        if (aborted) {
-            recvListener.dataReceived(this);
-        } else {
-            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() {
-        try {
-            socketChannel.close();
-        } catch (IOException e) {
-            e.printStackTrace();
-        }
-    }
-
-    @Override
-    public SelectionKey getSelectionKey() {
-        return key;
-    }
-
-    @Override
-    public UUID getJobId() {
-        return jobId;
-    }
-
-    @Override
-    public void setJobId(UUID jobId) {
-        this.jobId = jobId;
-    }
-
-    @Override
-    public UUID getStageId() {
-        return stageId;
-    }
-
-    @Override
-    public void setStageId(UUID stageId) {
-        this.stageId = stageId;
-    }
-
-    @Override
-    public void abort() {
-        aborted = true;
-    }
-
-    @Override
-    public boolean aborted() {
-        return aborted;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java
deleted file mode 100644
index 717144d..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java
+++ /dev/null
@@ -1,399 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.nc.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.HashSet;
-import java.util.Iterator;
-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.FrameConstants;
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
-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.context.IHyracksRootContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-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 IHyracksRootContext ctx;
-
-    private final Map<UUID, IDataReceiveListenerFactory> pendingConnectionReceivers;
-
-    private final ConnectionListenerThread connectionListenerThread;
-
-    private final DataListenerThread dataListenerThread;
-
-    private final IDataReceiveListener initialDataReceiveListener;
-
-    private final Set<IConnectionEntry> connections;
-
-    private volatile boolean stopped;
-
-    private ByteBuffer emptyFrame;
-
-    public ConnectionManager(IHyracksRootContext 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.allocateFrame();
-        emptyFrame.putInt(FrameHelper.getTupleCountOffset(ctx.getFrameSize()), 0);
-        connections = new HashSet<IConnectionEntry>();
-    }
-
-    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);
-    }
-
-    public synchronized void abortConnections(UUID jobId, UUID stageId) {
-        List<IConnectionEntry> abortConnections = new ArrayList<IConnectionEntry>();
-        synchronized (this) {
-            for (IConnectionEntry ce : connections) {
-                if (ce.getJobId().equals(jobId) && ce.getStageId().equals(stageId)) {
-                    abortConnections.add(ce);
-                }
-            }
-        }
-        dataListenerThread.addPendingAbortConnections(abortConnections);
-    }
-
-    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 {
-        }
-
-        @Override
-        public void flush() {
-        }
-    }
-
-    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> pendingNewSockets;
-        private List<IConnectionEntry> pendingAbortConnections;
-
-        public DataListenerThread() {
-            super("Hyracks Data Listener Thread");
-            try {
-                selector = Selector.open();
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
-            pendingNewSockets = new ArrayList<SocketChannel>();
-            pendingAbortConnections = new ArrayList<IConnectionEntry>();
-        }
-
-        synchronized void addSocketChannel(SocketChannel sc) throws IOException {
-            pendingNewSockets.add(sc);
-            selector.wakeup();
-        }
-
-        synchronized void addPendingAbortConnections(List<IConnectionEntry> abortConnections) {
-            pendingAbortConnections.addAll(abortConnections);
-            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 (!pendingNewSockets.isEmpty()) {
-                            for (SocketChannel sc : pendingNewSockets) {
-                                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");
-                                }
-                            }
-                            pendingNewSockets.clear();
-                        }
-                        if (!pendingAbortConnections.isEmpty()) {
-                            for (IConnectionEntry ce : pendingAbortConnections) {
-                                SelectionKey key = ce.getSelectionKey();
-                                ce.abort();
-                                ((ConnectionEntry) ce).dispatch(key);
-                                key.cancel();
-                                ce.close();
-                                synchronized (ConnectionManager.this) {
-                                    connections.remove(ce);
-                                }
-                            }
-                            pendingAbortConnections.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();
-                                    synchronized (ConnectionManager.this) {
-                                        connections.remove(entry);
-                                    }
-                                }
-                            }
-                        }
-                    }
-                } catch (Exception 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);
-                entry.setJobId(connectionReceiver.getJobId());
-                entry.setStageId(connectionReceiver.getStageId());
-                synchronized (ConnectionManager.this) {
-                    connections.add(entry);
-                }
-                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-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/DemuxDataReceiveListenerFactory.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/DemuxDataReceiveListenerFactory.java
deleted file mode 100644
index 32c8991..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/DemuxDataReceiveListenerFactory.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.nc.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.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public class DemuxDataReceiveListenerFactory implements IDataReceiveListenerFactory, IConnectionDemultiplexer,
-        IDataReceiveListener {
-    private static final Logger LOGGER = Logger.getLogger(DemuxDataReceiveListenerFactory.class.getName());
-
-    private final BitSet readyBits;
-    private final int frameSize;
-    private IConnectionEntry senders[];
-    private int openSenderCount;
-    private UUID jobId;
-    private UUID stageId;
-
-    public DemuxDataReceiveListenerFactory(IHyracksStageletContext ctx, UUID jobId, UUID stageId) {
-        frameSize = ctx.getFrameSize();
-        this.jobId = jobId;
-        this.stageId = stageId;
-        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;
-    }
-
-    @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 >= frameSize || entry.aborted()) {
-            if (LOGGER.isLoggable(Level.FINEST)) {
-                LOGGER.finest("NonDeterministicDataReceiveListener: frame received: sender = " + senderIndex);
-            }
-            SelectionKey key = entry.getSelectionKey();
-            if (key.isValid()) {
-                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;
-    }
-
-    @Override
-    public UUID getJobId() {
-        return jobId;
-    }
-
-    @Override
-    public UUID getStageId() {
-        return stageId;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOFuture.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOFuture.java
deleted file mode 100644
index bd6462d..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOFuture.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.nc.io;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public final class IOFuture {
-    private boolean complete;
-
-    private Exception exception;
-
-    public synchronized void reset() {
-        complete = false;
-        exception = null;
-    }
-
-    public synchronized void synchronize() throws HyracksDataException, InterruptedException {
-        while (!complete) {
-            try {
-                wait();
-            } catch (InterruptedException e) {
-                throw e;
-            }
-        }
-        if (exception != null) {
-            throw new HyracksDataException(exception);
-        }
-    }
-
-    public synchronized boolean isComplete() {
-        return complete;
-    }
-
-    public synchronized void notifySuccess() {
-        complete = true;
-        notifyAll();
-    }
-
-    public synchronized void notifyFailure(Exception e) {
-        complete = true;
-        exception = e;
-        notifyAll();
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/ManagedWorkspaceFileFactory.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/WorkspaceFileFactory.java
similarity index 75%
rename from hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/ManagedWorkspaceFileFactory.java
rename to hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/WorkspaceFileFactory.java
index c6be844..16421ab 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/ManagedWorkspaceFileFactory.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/WorkspaceFileFactory.java
@@ -20,24 +20,29 @@
 import edu.uci.ics.hyracks.api.resources.IDeallocatable;
 import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
 
-public final class ManagedWorkspaceFileFactory implements IWorkspaceFileFactory {
+public final class WorkspaceFileFactory implements IWorkspaceFileFactory {
     private final IDeallocatableRegistry registry;
     private final IOManager ioManager;
 
-    public ManagedWorkspaceFileFactory(IDeallocatableRegistry registry, IOManager ioManager) {
+    public WorkspaceFileFactory(IDeallocatableRegistry registry, IOManager ioManager) {
         this.registry = registry;
         this.ioManager = ioManager;
     }
 
     @Override
-    public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
+    public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
         final FileReference fRef = ioManager.createWorkspaceFile(prefix);
         registry.registerDeallocatable(new IDeallocatable() {
             @Override
             public void deallocate() {
-                fRef.getFile().delete();
+                fRef.delete();
             }
         });
         return fRef;
     }
+
+    @Override
+    public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException {
+        return ioManager.createWorkspaceFile(prefix);
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
new file mode 100644
index 0000000..b9eaf1f
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
@@ -0,0 +1,245 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.net;
+
+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.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.nc.partitions.IPartitionRequestListener;
+
+public class ConnectionManager {
+    private static final Logger LOGGER = Logger.getLogger(ConnectionManager.class.getName());
+
+    static final int INITIAL_MESSAGE_SIZE = 20;
+
+    private final IHyracksRootContext ctx;
+
+    private IPartitionRequestListener partitionRequestListener;
+
+    private final ServerSocketChannel serverChannel;
+
+    private volatile boolean stopped;
+
+    private final ConnectionListenerThread connectionListener;
+
+    private final DataListenerThread dataListener;
+
+    private final NetworkAddress networkAddress;
+
+    public ConnectionManager(IHyracksRootContext ctx, InetAddress inetAddress) throws IOException {
+        this.ctx = ctx;
+        serverChannel = ServerSocketChannel.open();
+        ServerSocket serverSocket = serverChannel.socket();
+        serverSocket.bind(new InetSocketAddress(inetAddress, 0));
+        stopped = false;
+        connectionListener = new ConnectionListenerThread();
+        dataListener = new DataListenerThread();
+        networkAddress = new NetworkAddress(serverSocket.getInetAddress(), serverSocket.getLocalPort());
+
+    }
+
+    public void setPartitionRequestListener(IPartitionRequestListener partitionRequestListener) {
+        this.partitionRequestListener = partitionRequestListener;
+    }
+
+    public void start() {
+        connectionListener.start();
+        dataListener.start();
+    }
+
+    public void stop() {
+        try {
+            stopped = true;
+            serverChannel.close();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+
+    public void connect(INetworkChannel channel) throws IOException {
+        dataListener.addOutgoingConnection(channel);
+    }
+
+    private final class ConnectionListenerThread extends Thread {
+        public ConnectionListenerThread() {
+            super("Hyracks NC Connection Listener");
+            setDaemon(true);
+        }
+
+        @Override
+        public void run() {
+            while (!stopped) {
+                try {
+                    SocketChannel sc = serverChannel.accept();
+                    dataListener.addIncomingConnection(sc);
+                } catch (AsynchronousCloseException e) {
+                    // do nothing
+                    if (!stopped) {
+                        e.printStackTrace();
+                    }
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+
+    private final class DataListenerThread extends Thread {
+        private Selector selector;
+
+        private final List<SocketChannel> pendingIncomingConnections;
+        private final Set<SocketChannel> pendingNegotiations;
+        private final List<INetworkChannel> pendingOutgoingConnections;
+
+        public DataListenerThread() {
+            super("Hyracks Data Listener Thread");
+            setDaemon(true);
+            try {
+                selector = Selector.open();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+            pendingIncomingConnections = new ArrayList<SocketChannel>();
+            pendingNegotiations = new HashSet<SocketChannel>();
+            pendingOutgoingConnections = new ArrayList<INetworkChannel>();
+        }
+
+        synchronized void addIncomingConnection(SocketChannel sc) throws IOException {
+            pendingIncomingConnections.add(sc);
+            selector.wakeup();
+        }
+
+        synchronized void addOutgoingConnection(INetworkChannel channel) throws IOException {
+            pendingOutgoingConnections.add(channel);
+            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 (!pendingIncomingConnections.isEmpty()) {
+                            for (SocketChannel sc : pendingIncomingConnections) {
+                                sc.configureBlocking(false);
+                                SelectionKey scKey = sc.register(selector, SelectionKey.OP_READ);
+                                ByteBuffer buffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
+                                scKey.attach(buffer);
+                                pendingNegotiations.add(sc);
+                            }
+                            pendingIncomingConnections.clear();
+                        }
+                        if (!pendingOutgoingConnections.isEmpty()) {
+                            for (INetworkChannel nc : pendingOutgoingConnections) {
+                                SocketChannel sc = SocketChannel.open();
+                                sc.configureBlocking(false);
+                                SelectionKey scKey = sc.register(selector, 0);
+                                scKey.attach(nc);
+                                nc.setSelectionKey(scKey);
+                                nc.notifyConnectionManagerRegistration();
+                            }
+                            pendingOutgoingConnections.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();
+                                SocketChannel sc = (SocketChannel) key.channel();
+                                if (pendingNegotiations.contains(sc)) {
+                                    if (key.isReadable()) {
+                                        ByteBuffer buffer = (ByteBuffer) key.attachment();
+                                        sc.read(buffer);
+                                        buffer.flip();
+                                        if (buffer.remaining() >= INITIAL_MESSAGE_SIZE) {
+                                            PartitionId pid = readInitialMessage(buffer);
+                                            pendingNegotiations.remove(sc);
+                                            key.interestOps(0);
+                                            NetworkOutputChannel channel = new NetworkOutputChannel(ctx, 5);
+                                            channel.setSelectionKey(key);
+                                            key.attach(channel);
+                                            try {
+                                                partitionRequestListener.registerPartitionRequest(pid, channel);
+                                            } catch (HyracksException e) {
+                                                key.cancel();
+                                                sc.close();
+                                            }
+                                        } else {
+                                            buffer.compact();
+                                        }
+                                    }
+                                } else {
+                                    INetworkChannel channel = (INetworkChannel) key.attachment();
+                                    boolean close = false;
+                                    try {
+                                        close = channel.dispatchNetworkEvent();
+                                    } catch (IOException e) {
+                                        e.printStackTrace();
+                                        close = true;
+                                    }
+                                    if (close) {
+                                        key.cancel();
+                                        sc.close();
+                                    }
+                                }
+                            }
+                        }
+                    }
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+
+        private PartitionId readInitialMessage(ByteBuffer buffer) {
+            JobId jobId = new JobId(buffer.getLong());
+            ConnectorDescriptorId cdid = new ConnectorDescriptorId(buffer.getInt());
+            int senderIndex = buffer.getInt();
+            int receiverIndex = buffer.getInt();
+            return new PartitionId(jobId, cdid, senderIndex, receiverIndex);
+        }
+    }
+
+    public NetworkAddress getNetworkAddress() {
+        return networkAddress;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java
new file mode 100644
index 0000000..61cd91f
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java
@@ -0,0 +1,19 @@
+package edu.uci.ics.hyracks.control.nc.net;
+
+import java.io.IOException;
+import java.net.SocketAddress;
+import java.nio.channels.SelectionKey;
+
+public interface INetworkChannel {
+    public boolean dispatchNetworkEvent() throws IOException;
+
+    public void setSelectionKey(SelectionKey key);
+
+    public SelectionKey getSelectionKey();
+
+    public SocketAddress getRemoteAddress();
+
+    public void abort();
+
+    public void notifyConnectionManagerRegistration() throws IOException;
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
new file mode 100644
index 0000000..58f7088
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.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.control.nc.net;
+
+import java.io.IOException;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayDeque;
+import java.util.Queue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class NetworkInputChannel implements IInputChannel, INetworkChannel {
+    private static final Logger LOGGER = Logger.getLogger(NetworkInputChannel.class.getName());
+
+    private final ConnectionManager connectionManager;
+
+    private final SocketAddress remoteAddress;
+
+    private final PartitionId partitionId;
+
+    private final Queue<ByteBuffer> emptyQueue;
+
+    private final Queue<ByteBuffer> fullQueue;
+
+    private SocketChannel socketChannel;
+
+    private SelectionKey key;
+
+    private ByteBuffer currentBuffer;
+
+    private boolean eos;
+
+    private boolean aborted;
+
+    private IInputChannelMonitor monitor;
+
+    private Object attachment;
+
+    private ByteBuffer writeBuffer;
+
+    public NetworkInputChannel(IHyracksRootContext ctx, ConnectionManager connectionManager,
+            SocketAddress remoteAddress, PartitionId partitionId, int nBuffers) {
+        this.connectionManager = connectionManager;
+        this.remoteAddress = remoteAddress;
+        this.partitionId = partitionId;
+        this.emptyQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+        for (int i = 0; i < nBuffers; ++i) {
+            emptyQueue.add(ctx.allocateFrame());
+        }
+        fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+        aborted = false;
+        eos = false;
+    }
+
+    @Override
+    public void registerMonitor(IInputChannelMonitor monitor) throws HyracksException {
+        this.monitor = monitor;
+    }
+
+    @Override
+    public void setAttachment(Object attachment) {
+        this.attachment = attachment;
+    }
+
+    @Override
+    public Object getAttachment() {
+        return attachment;
+    }
+
+    @Override
+    public synchronized ByteBuffer getNextBuffer() {
+        return fullQueue.poll();
+    }
+
+    @Override
+    public synchronized void recycleBuffer(ByteBuffer buffer) {
+        buffer.clear();
+        emptyQueue.add(buffer);
+        if (!eos && !aborted) {
+            int ops = key.interestOps();
+            if ((ops & SelectionKey.OP_READ) == 0) {
+                key.interestOps(ops | SelectionKey.OP_READ);
+                key.selector().wakeup();
+                if (currentBuffer == null) {
+                    currentBuffer = emptyQueue.poll();
+                }
+            }
+        }
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        currentBuffer = emptyQueue.poll();
+        try {
+            connectionManager.connect(this);
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+
+    }
+
+    @Override
+    public synchronized boolean dispatchNetworkEvent() throws IOException {
+        if (aborted) {
+            eos = true;
+            monitor.notifyEndOfStream(this);
+            return true;
+        }
+        if (key.isConnectable()) {
+            if (socketChannel.finishConnect()) {
+                key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT);
+                prepareForWrite();
+            }
+        } else if (key.isWritable()) {
+            socketChannel.write(writeBuffer);
+            if (writeBuffer.remaining() == 0) {
+                key.interestOps(SelectionKey.OP_READ);
+            }
+        } else if (key.isReadable()) {
+            if (LOGGER.isLoggable(Level.FINER)) {
+                LOGGER.finer("Before read: " + currentBuffer.position() + " " + currentBuffer.limit());
+            }
+            int bytesRead = socketChannel.read(currentBuffer);
+            if (bytesRead < 0) {
+                eos = true;
+                monitor.notifyEndOfStream(this);
+                return true;
+            }
+            if (LOGGER.isLoggable(Level.FINER)) {
+                LOGGER.finer("After read: " + currentBuffer.position() + " " + currentBuffer.limit());
+            }
+            currentBuffer.flip();
+            int dataLen = currentBuffer.remaining();
+            if (dataLen >= currentBuffer.capacity() || aborted()) {
+                if (LOGGER.isLoggable(Level.FINEST)) {
+                    LOGGER.finest("NetworkInputChannel: frame received: sender = " + partitionId.getSenderIndex());
+                }
+                if (currentBuffer.getInt(FrameHelper.getTupleCountOffset(currentBuffer.capacity())) == 0) {
+                    eos = true;
+                    monitor.notifyEndOfStream(this);
+                    return true;
+                }
+                fullQueue.add(currentBuffer);
+                currentBuffer = emptyQueue.poll();
+                if (currentBuffer == null && key.isValid()) {
+                    int ops = key.interestOps();
+                    key.interestOps(ops & ~SelectionKey.OP_READ);
+                }
+                monitor.notifyDataAvailability(this, 1);
+                return false;
+            }
+            currentBuffer.compact();
+        }
+        return false;
+    }
+
+    private void prepareForConnect() {
+        key.interestOps(SelectionKey.OP_CONNECT);
+    }
+
+    private void prepareForWrite() {
+        writeBuffer = ByteBuffer.allocate(ConnectionManager.INITIAL_MESSAGE_SIZE);
+        writeBuffer.putLong(partitionId.getJobId().getId());
+        writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
+        writeBuffer.putInt(partitionId.getSenderIndex());
+        writeBuffer.putInt(partitionId.getReceiverIndex());
+        writeBuffer.flip();
+
+        key.interestOps(SelectionKey.OP_WRITE);
+    }
+
+    @Override
+    public void setSelectionKey(SelectionKey key) {
+        this.key = key;
+        socketChannel = (SocketChannel) key.channel();
+    }
+
+    @Override
+    public SocketAddress getRemoteAddress() {
+        return remoteAddress;
+    }
+
+    @Override
+    public SelectionKey getSelectionKey() {
+        return key;
+    }
+
+    public PartitionId getPartitionId() {
+        return partitionId;
+    }
+
+    public void abort() {
+        aborted = true;
+    }
+
+    public boolean aborted() {
+        return aborted;
+    }
+
+    @Override
+    public void notifyConnectionManagerRegistration() throws IOException {
+        if (socketChannel.connect(remoteAddress)) {
+            prepareForWrite();
+        } else {
+            prepareForConnect();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
new file mode 100644
index 0000000..74542b3
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
@@ -0,0 +1,171 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.net;
+
+import java.io.IOException;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class NetworkOutputChannel implements INetworkChannel, IFrameWriter {
+    private final IHyracksRootContext ctx;
+
+    private final Queue<ByteBuffer> emptyQueue;
+
+    private final Queue<ByteBuffer> fullQueue;
+
+    private SelectionKey key;
+
+    private boolean aborted;
+
+    private boolean eos;
+
+    private boolean eosSent;
+
+    private ByteBuffer currentBuffer;
+
+    public NetworkOutputChannel(IHyracksRootContext ctx, int nBuffers) {
+        this.ctx = ctx;
+        emptyQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+        for (int i = 0; i < nBuffers; ++i) {
+            emptyQueue.add(ctx.allocateFrame());
+        }
+        fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+    }
+
+    @Override
+    public synchronized boolean dispatchNetworkEvent() throws IOException {
+        if (aborted) {
+            eos = true;
+            return true;
+        } else if (key.isWritable()) {
+            while (true) {
+                if (currentBuffer == null) {
+                    if (eosSent) {
+                        return true;
+                    }
+                    currentBuffer = fullQueue.poll();
+                    if (currentBuffer == null) {
+                        if (eos) {
+                            currentBuffer = emptyQueue.poll();
+                            currentBuffer.clear();
+                            currentBuffer.putInt(FrameHelper.getTupleCountOffset(ctx.getFrameSize()), 0);
+                            eosSent = true;
+                        } else {
+                            key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE);
+                            return false;
+                        }
+                    }
+                }
+                int bytesWritten = ((SocketChannel) key.channel()).write(currentBuffer);
+                if (bytesWritten < 0) {
+                    eos = true;
+                    return true;
+                }
+                if (currentBuffer.remaining() == 0) {
+                    emptyQueue.add(currentBuffer);
+                    notifyAll();
+                    currentBuffer = null;
+                    if (eosSent) {
+                        return true;
+                    }
+                } else {
+                    return false;
+                }
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public void setSelectionKey(SelectionKey key) {
+        this.key = key;
+    }
+
+    @Override
+    public SelectionKey getSelectionKey() {
+        return key;
+    }
+
+    @Override
+    public SocketAddress getRemoteAddress() {
+        return ((SocketChannel) key.channel()).socket().getRemoteSocketAddress();
+    }
+
+    @Override
+    public synchronized void abort() {
+        aborted = true;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        currentBuffer = null;
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        ByteBuffer destBuffer = null;
+        synchronized (this) {
+            if (aborted) {
+                throw new HyracksDataException("Connection has been aborted");
+            }
+            while (true) {
+                destBuffer = emptyQueue.poll();
+                if (destBuffer != null) {
+                    break;
+                }
+                try {
+                    wait();
+                } catch (InterruptedException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        }
+        buffer.position(0);
+        buffer.limit(destBuffer.capacity());
+        destBuffer.clear();
+        destBuffer.put(buffer);
+        destBuffer.flip();
+        synchronized (this) {
+            fullQueue.add(destBuffer);
+        }
+        key.interestOps(SelectionKey.OP_WRITE);
+        key.selector().wakeup();
+    }
+
+    @Override
+    public void flush() throws HyracksDataException {
+
+    }
+
+    @Override
+    public synchronized void close() throws HyracksDataException {
+        eos = true;
+        key.interestOps(SelectionKey.OP_WRITE);
+        key.selector().wakeup();        
+    }
+
+    @Override
+    public void notifyConnectionManagerRegistration() throws IOException {
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/IPartitionRequestListener.java
similarity index 67%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
copy to hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/IPartitionRequestListener.java
index 45c589a..20ed49c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/IPartitionRequestListener.java
@@ -12,11 +12,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.control.nc.partitions;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
 
-public interface IEndpointDataWriterFactory {
-    public IFrameWriter createFrameWriter(int index) throws HyracksDataException;
+public interface IPartitionRequestListener {
+    public void registerPartitionRequest(PartitionId partitionId, IFrameWriter writer) throws HyracksException;
 }
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartition.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartition.java
new file mode 100644
index 0000000..97f82d9
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartition.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.control.nc.partitions;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.Executor;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileHandle;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.partitions.IPartition;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+
+public class MaterializedPartition implements IPartition {
+    private final IHyracksRootContext ctx;
+
+    private final FileReference partitionFile;
+
+    private final Executor executor;
+
+    private final IOManager ioManager;
+
+    public MaterializedPartition(IHyracksRootContext ctx, FileReference partitionFile, Executor executor,
+            IOManager ioManager) {
+        this.ctx = ctx;
+        this.partitionFile = partitionFile;
+        this.executor = executor;
+        this.ioManager = ioManager;
+    }
+
+    @Override
+    public void deallocate() {
+        partitionFile.delete();
+    }
+
+    @Override
+    public void writeTo(final IFrameWriter writer) {
+        executor.execute(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    FileHandle fh = ioManager.open(partitionFile, IIOManager.FileReadWriteMode.READ_ONLY,
+                            IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+                    try {
+                        writer.open();
+                        try {
+                            long offset = 0;
+                            ByteBuffer buffer = ctx.allocateFrame();
+                            while (true) {
+                                buffer.clear();
+                                long size = ioManager.syncRead(fh, offset, buffer);
+                                if (size < 0) {
+                                    break;
+                                } else if (size < buffer.capacity()) {
+                                    throw new HyracksDataException("Premature end of file");
+                                }
+                                offset += size;
+                                buffer.flip();
+                                writer.nextFrame(buffer);
+                            }
+                        } finally {
+                            writer.close();
+                        }
+                    } finally {
+                        ioManager.close(fh);
+                    }
+                } catch (HyracksDataException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        });
+    }
+
+    @Override
+    public boolean isReusable() {
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
new file mode 100644
index 0000000..971f2b6
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
@@ -0,0 +1,132 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.partitions;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class MaterializedPartitionInputChannel implements IInputChannel {
+    private final Queue<ByteBuffer> emptyQueue;
+
+    private final Queue<ByteBuffer> fullQueue;
+
+    private final PartitionId pid;
+
+    private final PartitionManager manager;
+
+    private final FrameWriter writer;
+
+    private IInputChannelMonitor monitor;
+
+    private Object attachment;
+
+    public MaterializedPartitionInputChannel(IHyracksRootContext ctx, int nBuffers, PartitionId pid,
+            PartitionManager manager) {
+        this.emptyQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+        for (int i = 0; i < nBuffers; ++i) {
+            emptyQueue.add(ctx.allocateFrame());
+        }
+        fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+        this.pid = pid;
+        this.manager = manager;
+        writer = new FrameWriter();
+    }
+
+    @Override
+    public void registerMonitor(IInputChannelMonitor monitor) throws HyracksException {
+        this.monitor = monitor;
+    }
+
+    @Override
+    public void setAttachment(Object attachment) {
+        this.attachment = attachment;
+    }
+
+    @Override
+    public Object getAttachment() {
+        return attachment;
+    }
+
+    @Override
+    public ByteBuffer getNextBuffer() {
+        return fullQueue.poll();
+    }
+
+    @Override
+    public void recycleBuffer(ByteBuffer buffer) {
+        buffer.clear();
+        synchronized (this) {
+            emptyQueue.add(buffer);
+            notifyAll();
+        }
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        MaterializedPartition partition = (MaterializedPartition) manager.getPartition(pid);
+        partition.writeTo(writer);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+
+    }
+
+    private class FrameWriter implements IFrameWriter {
+        @Override
+        public void open() throws HyracksDataException {
+
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            synchronized (MaterializedPartitionInputChannel.this) {
+                while (emptyQueue.isEmpty()) {
+                    try {
+                        MaterializedPartitionInputChannel.this.wait();
+                    } catch (InterruptedException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+                ByteBuffer destFrame = emptyQueue.poll();
+                buffer.position(0);
+                buffer.limit(buffer.capacity());
+                destFrame.clear();
+                destFrame.put(buffer);
+                fullQueue.add(destFrame);
+                monitor.notifyDataAvailability(MaterializedPartitionInputChannel.this, 1);
+            }
+        }
+
+        @Override
+        public void flush() throws HyracksDataException {
+
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            monitor.notifyEndOfStream(MaterializedPartitionInputChannel.this);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionWriter.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionWriter.java
new file mode 100644
index 0000000..db2e405
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionWriter.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.control.nc.partitions;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.Executor;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileHandle;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+
+public class MaterializedPartitionWriter implements IFrameWriter {
+    protected final IHyracksRootContext ctx;
+
+    protected final PartitionManager manager;
+
+    protected final PartitionId pid;
+
+    protected final TaskAttemptId taId;
+
+    protected final Executor executor;
+
+    private FileReference fRef;
+
+    private FileHandle handle;
+
+    private long size;
+
+    public MaterializedPartitionWriter(IHyracksRootContext ctx, PartitionManager manager, PartitionId pid,
+            TaskAttemptId taId, Executor executor) {
+        this.ctx = ctx;
+        this.manager = manager;
+        this.pid = pid;
+        this.taId = taId;
+        this.executor = executor;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        fRef = manager.getFileFactory().createUnmanagedWorkspaceFile(pid.toString());
+        handle = ctx.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+        size = 0;
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        size += ctx.getIOManager().syncWrite(handle, size, buffer);
+    }
+
+    @Override
+    public void flush() throws HyracksDataException {
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        ctx.getIOManager().close(handle);
+        manager.registerPartition(pid, taId,
+                new MaterializedPartition(ctx, fRef, executor, (IOManager) ctx.getIOManager()),
+                PartitionState.COMMITTED);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
new file mode 100644
index 0000000..baa50a1
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.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.control.nc.partitions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.IPartition;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
+
+public class PartitionManager implements IPartitionRequestListener {
+    private final NodeControllerService ncs;
+
+    private final Map<PartitionId, List<IPartition>> partitionMap;
+
+    private final DefaultDeallocatableRegistry deallocatableRegistry;
+
+    private final IWorkspaceFileFactory fileFactory;
+
+    public PartitionManager(NodeControllerService ncs) {
+        this.ncs = ncs;
+        partitionMap = new HashMap<PartitionId, List<IPartition>>();
+        deallocatableRegistry = new DefaultDeallocatableRegistry();
+        fileFactory = new WorkspaceFileFactory(deallocatableRegistry, (IOManager) ncs.getRootContext().getIOManager());
+    }
+
+    public void registerPartition(PartitionId pid, TaskAttemptId taId, IPartition partition, PartitionState state)
+            throws HyracksDataException {
+        synchronized (this) {
+            List<IPartition> pList = partitionMap.get(pid);
+            if (pList == null) {
+                pList = new ArrayList<IPartition>();
+                partitionMap.put(pid, pList);
+            }
+            pList.add(partition);
+        }
+        updatePartitionState(pid, taId, partition, state);
+    }
+
+    public void updatePartitionState(PartitionId pid, TaskAttemptId taId, IPartition partition, PartitionState state)
+            throws HyracksDataException {
+        PartitionDescriptor desc = new PartitionDescriptor(pid, ncs.getId(), taId, partition.isReusable());
+        desc.setState(state);
+        try {
+            ncs.getClusterController().registerPartitionProvider(desc);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public synchronized IPartition getPartition(PartitionId pid) {
+        return partitionMap.get(pid).get(0);
+    }
+
+    public synchronized void unregisterPartitions(JobId jobId) {
+        for (Iterator<Map.Entry<PartitionId, List<IPartition>>> i = partitionMap.entrySet().iterator(); i.hasNext();) {
+            Map.Entry<PartitionId, List<IPartition>> e = i.next();
+            PartitionId pid = e.getKey();
+            if (jobId.equals(pid.getJobId())) {
+                for (IPartition p : e.getValue()) {
+                    p.deallocate();
+                }
+                i.remove();
+            }
+        }
+    }
+
+    @Override
+    public synchronized void registerPartitionRequest(PartitionId partitionId, IFrameWriter writer)
+            throws HyracksException {
+        List<IPartition> pList = partitionMap.get(partitionId);
+        if (pList != null && !pList.isEmpty()) {
+            IPartition partition = pList.get(0);
+            partition.writeTo(writer);
+            if (!partition.isReusable()) {
+                partitionMap.remove(partitionId);
+            }
+        } else {
+            throw new HyracksException("Request for unknown partition " + partitionId);
+        }
+    }
+
+    public IWorkspaceFileFactory getFileFactory() {
+        return fileFactory;
+    }
+
+    public void close() {
+        deallocatableRegistry.close();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
new file mode 100644
index 0000000..1a9fb53
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.partitions;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.partitions.IPartition;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
+
+public class PipelinedPartition implements IFrameWriter, IPartition {
+    private final PartitionManager manager;
+
+    private final PartitionId pid;
+
+    private final TaskAttemptId taId;
+
+    private IFrameWriter delegate;
+
+    public PipelinedPartition(PartitionManager manager, PartitionId pid, TaskAttemptId taId) {
+        this.manager = manager;
+        this.pid = pid;
+        this.taId = taId;
+    }
+
+    @Override
+    public boolean isReusable() {
+        return false;
+    }
+
+    @Override
+    public void deallocate() {
+        // do nothing
+    }
+
+    @Override
+    public synchronized void writeTo(IFrameWriter writer) {
+        delegate = writer;
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void open() throws HyracksDataException {
+        manager.registerPartition(pid, taId, this, PartitionState.STARTED);
+        while (delegate == null) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        delegate.open();
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        delegate.nextFrame(buffer);
+    }
+
+    @Override
+    public void flush() throws HyracksDataException {
+        delegate.flush();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        manager.updatePartitionState(pid, taId, this, PartitionState.COMMITTED);
+        delegate.close();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java
new file mode 100644
index 0000000..c108bef
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java
@@ -0,0 +1,160 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.partitions;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.Executor;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class ReceiveSideMaterializingCollector implements IPartitionCollector {
+    private final IHyracksRootContext ctx;
+
+    private PartitionManager manager;
+
+    private final IPartitionCollector delegate;
+
+    private final TaskAttemptId taId;
+
+    private final Executor executor;
+
+    public ReceiveSideMaterializingCollector(IHyracksRootContext ctx, PartitionManager manager,
+            IPartitionCollector collector, TaskAttemptId taId, Executor executor) {
+        this.ctx = ctx;
+        this.manager = manager;
+        this.delegate = collector;
+        this.taId = taId;
+        this.executor = executor;
+    }
+
+    @Override
+    public JobId getJobId() {
+        return delegate.getJobId();
+    }
+
+    @Override
+    public ConnectorDescriptorId getConnectorId() {
+        return delegate.getConnectorId();
+    }
+
+    @Override
+    public int getReceiverIndex() {
+        return delegate.getReceiverIndex();
+    }
+
+    @Override
+    public void open() throws HyracksException {
+        delegate.open();
+    }
+
+    @Override
+    public void addPartitions(Collection<PartitionChannel> partitions) throws HyracksException {
+        for (final PartitionChannel pc : partitions) {
+            PartitionWriter writer = new PartitionWriter(pc);
+            executor.execute(writer);
+        }
+    }
+
+    private class PartitionWriter implements Runnable, IInputChannelMonitor {
+        private PartitionChannel pc;
+
+        private int nAvailableFrames;
+
+        private boolean eos;
+
+        public PartitionWriter(PartitionChannel pc) {
+            this.pc = pc;
+            nAvailableFrames = 0;
+            eos = false;
+        }
+
+        @Override
+        public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
+            nAvailableFrames += nFrames;
+            notifyAll();
+        }
+
+        @Override
+        public synchronized void notifyEndOfStream(IInputChannel channel) {
+            eos = true;
+            notifyAll();
+        }
+
+        @Override
+        public synchronized void run() {
+            PartitionId pid = pc.getPartitionId();
+            MaterializedPartitionWriter mpw = new MaterializedPartitionWriter(ctx, manager, pid, taId, executor);
+            IInputChannel channel = pc.getInputChannel();
+            try {
+                channel.registerMonitor(this);
+                channel.open();
+                mpw.open();
+                while (true) {
+                    if (nAvailableFrames > 0) {
+                        ByteBuffer buffer = channel.getNextBuffer();
+                        --nAvailableFrames;
+                        mpw.nextFrame(buffer);
+                        channel.recycleBuffer(buffer);
+                    } else if (eos) {
+                        break;
+                    } else {
+                        try {
+                            wait();
+                        } catch (InterruptedException e) {
+                            e.printStackTrace();
+                        }
+                    }
+                }
+                mpw.close();
+                channel.close();
+                delegate.addPartitions(Collections.singleton(new PartitionChannel(pid,
+                        new MaterializedPartitionInputChannel(ctx, 5, pid, manager))));
+            } catch (HyracksException e) {
+            }
+        }
+    }
+
+    @Override
+    public IFrameReader getReader() throws HyracksException {
+        return delegate.getReader();
+    }
+
+    @Override
+    public void close() throws HyracksException {
+        delegate.close();
+    }
+
+    @Override
+    public Collection<PartitionId> getRequiredPartitionIds() throws HyracksException {
+        return delegate.getRequiredPartitionIds();
+    }
+
+    @Override
+    public void abort() {
+        delegate.abort();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java
new file mode 100644
index 0000000..96fcf75
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.profiling;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
+
+public class ConnectorReceiverProfilingFrameReader implements IFrameReader {
+    private final IFrameReader reader;
+    private final ICounter openCounter;
+    private final ICounter closeCounter;
+    private final ICounter frameCounter;
+
+    public ConnectorReceiverProfilingFrameReader(IHyracksTaskContext ctx, IFrameReader reader,
+            ConnectorDescriptorId cdId, int receiverIndex) {
+        this.reader = reader;
+        this.openCounter = ctx.getCounterContext().getCounter(cdId + ".receiver." + receiverIndex + ".open", true);
+        this.closeCounter = ctx.getCounterContext().getCounter(cdId + ".receiver." + receiverIndex + ".close", true);
+        this.frameCounter = ctx.getCounterContext()
+                .getCounter(cdId + ".receiver." + receiverIndex + ".nextFrame", true);
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        reader.open();
+        openCounter.update(1);
+    }
+
+    @Override
+    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        boolean status = reader.nextFrame(buffer);
+        if (status) {
+            frameCounter.update(1);
+        }
+        return status;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        reader.close();
+        closeCounter.update(1);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorSenderProfilingFrameWriter.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorSenderProfilingFrameWriter.java
new file mode 100644
index 0000000..c64e187
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorSenderProfilingFrameWriter.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.profiling;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
+
+public class ConnectorSenderProfilingFrameWriter implements IFrameWriter {
+    private final IFrameWriter writer;
+    private final ICounter openCounter;
+    private final ICounter closeCounter;
+    private final ICounter frameCounter;
+
+    public ConnectorSenderProfilingFrameWriter(IHyracksTaskContext ctx, IFrameWriter writer,
+            ConnectorDescriptorId cdId, int senderIndex, int receiverIndex) {
+        this.writer = writer;
+        this.openCounter = ctx.getCounterContext().getCounter(
+                cdId + ".sender." + senderIndex + "." + receiverIndex + ".open", true);
+        this.closeCounter = ctx.getCounterContext().getCounter(
+                cdId + ".sender." + senderIndex + "." + receiverIndex + ".close", true);
+        this.frameCounter = ctx.getCounterContext().getCounter(
+                cdId + ".sender." + senderIndex + "." + receiverIndex + ".nextFrame", true);
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        writer.open();
+        openCounter.update(1);
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        frameCounter.update(1);
+        writer.nextFrame(buffer);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        closeCounter.update(1);
+        writer.close();
+    }
+
+    @Override
+    public void flush() throws HyracksDataException {
+        writer.flush();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/OperatorRunnable.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/OperatorRunnable.java
deleted file mode 100644
index 8edd992..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/OperatorRunnable.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.nc.runtime;
-
-import java.nio.ByteBuffer;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Semaphore;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-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;
-
-public class OperatorRunnable implements Runnable {
-    private final IHyracksStageletContext ctx;
-    private final IOperatorNodePushable opNode;
-    private final int nInputs;
-    private final Executor executor;
-    private IFrameReader[] readers;
-    private volatile boolean abort;
-
-    public OperatorRunnable(IHyracksStageletContext ctx, IOperatorNodePushable opNode, int nInputs, Executor executor) {
-        this.ctx = ctx;
-        this.opNode = opNode;
-        this.nInputs = nInputs;
-        this.executor = executor;
-        readers = new IFrameReader[nInputs];
-    }
-
-    public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
-        opNode.setOutputFrameWriter(index, writer, recordDesc);
-    }
-
-    public void setFrameReader(int inputIdx, IFrameReader reader) {
-        this.readers[inputIdx] = reader;
-    }
-
-    public void abort() {
-        abort = true;
-    }
-
-    @Override
-    public void run() {
-        try {
-            opNode.initialize();
-            if (nInputs > 0) {
-                final Semaphore sem = new Semaphore(nInputs - 1);
-                for (int i = 1; i < nInputs; ++i) {
-                    final IFrameReader reader = readers[i];
-                    final IFrameWriter writer = opNode.getInputFrameWriter(i);
-                    sem.acquire();
-                    executor.execute(new Runnable() {
-                        public void run() {
-                            try {
-                                pushFrames(reader, writer);
-                            } catch (HyracksDataException e) {
-                            } finally {
-                                sem.release();
-                            }
-                        }
-                    });
-                }
-                try {
-                    pushFrames(readers[0], opNode.getInputFrameWriter(0));
-                } finally {
-                    sem.acquire(nInputs - 1);
-                }
-            }
-            opNode.deinitialize();
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private void pushFrames(IFrameReader reader, IFrameWriter writer) throws HyracksDataException {
-        ByteBuffer buffer = ctx.allocateFrame();
-        writer.open();
-        reader.open();
-        while (reader.nextFrame(buffer)) {
-            if (abort) {
-                break;
-            }
-            buffer.flip();
-            writer.nextFrame(buffer);
-            buffer.compact();
-        }
-        reader.close();
-        writer.close();
-    }
-
-    @Override
-    public String toString() {
-        return "OperatorRunnable[" + opNode.getDisplayName() + "]";
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/NonDeterministicFrameReader.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/NonDeterministicFrameReader.java
deleted file mode 100644
index 49d3db7..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/NonDeterministicFrameReader.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.comm;
-
-import java.nio.ByteBuffer;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
-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.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public class NonDeterministicFrameReader implements IFrameReader {
-    private static final Logger LOGGER = Logger.getLogger(NonDeterministicFrameReader.class.getName());
-
-    private final IHyracksStageletContext ctx;
-    private final IConnectionDemultiplexer demux;
-    private int lastReadSender;
-    private boolean eos;
-
-    public NonDeterministicFrameReader(IHyracksStageletContext 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);
-            if (entry.aborted()) {
-                eos = true;
-                return false;
-            }
-            lastReadSender = (Integer) entry.getAttachment();
-            ByteBuffer netBuffer = entry.getReadBuffer();
-            int tupleCount = netBuffer.getInt(FrameHelper.getTupleCountOffset(ctx.getFrameSize()));
-            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-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/SortMergeFrameReader.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/SortMergeFrameReader.java
deleted file mode 100644
index 7be9b08..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/SortMergeFrameReader.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.comm;
-
-import java.nio.ByteBuffer;
-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.FrameHelper;
-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.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileHandle;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
-
-public class SortMergeFrameReader implements IFrameReader {
-    private static final Logger LOGGER = Logger.getLogger(SortMergeFrameReader.class.getName());
-
-    private final IHyracksStageletContext 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(IHyracksStageletContext 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.getFrameSize());
-        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.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 FileReference fRef;
-        private final FileHandle fHandle;
-        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;
-            fRef = ctx.createWorkspaceFile(SortMergeFrameReader.class.getName());
-            fHandle = ctx.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
-                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
-            frame = ctx.allocateFrame();
-            accessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-            readFP = 0;
-            writeFP = 0;
-            eof = false;
-        }
-
-        public void close() throws HyracksDataException {
-            ctx.getIOManager().close(fHandle);
-        }
-
-        private void write(ByteBuffer frame) throws HyracksDataException {
-            int sz = ctx.getIOManager().syncWrite(fHandle, writeFP, frame);
-            writeFP += sz;
-        }
-
-        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 {
-            frame.clear();
-            while (!eof && readFP >= writeFP) {
-                spoolRuns(runId);
-            }
-            if (eof && readFP >= writeFP) {
-                return false;
-            }
-            int sz = ctx.getIOManager().syncRead(fHandle, readFP, frame);
-            readFP += sz;
-            return true;
-        }
-
-        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.getFrameSize()));
-            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-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
index d2a3c53..f4862d8 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
@@ -17,7 +17,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -33,7 +33,7 @@
 
     private final FrameDeserializer frameDeserializer;
 
-    public FrameDeserializingDataReader(IHyracksStageletContext ctx, IFrameReader frameReader,
+    public FrameDeserializingDataReader(IHyracksTaskContext ctx, IFrameReader frameReader,
             RecordDescriptor recordDescriptor) {
         buffer = ctx.allocateFrame();
         this.frameReader = frameReader;
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
index 9edd3bf..7c7979b 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
@@ -19,7 +19,7 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -39,8 +39,7 @@
 
     private boolean open;
 
-    public SerializingDataWriter(IHyracksStageletContext ctx, RecordDescriptor recordDescriptor,
-            IFrameWriter frameWriter) {
+    public SerializingDataWriter(IHyracksTaskContext ctx, RecordDescriptor recordDescriptor, IFrameWriter frameWriter) {
         buffer = ctx.allocateFrame();
         tb = new ArrayTupleBuilder(recordDescriptor.getFields().length);
         this.recordDescriptor = recordDescriptor;
diff --git a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
index fd549b3..14c034a 100644
--- a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
+++ b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
@@ -34,7 +34,7 @@
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.util.ReflectionUtils;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -367,7 +367,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
 
         JobConf conf = getJobConf();
@@ -398,7 +398,7 @@
                             (Class<? extends Writable>) oldReader.createKey().getClass(),
                             (Class<? extends Writable>) oldReader.createValue().getClass());
                 }
-                return createSelfReadingMapper(ctx, env, recordDescriptor, partition);
+                return createSelfReadingMapper(ctx, recordDescriptor, partition);
             } else {
                 return new DeserializedOperatorNodePushable(ctx, new MapperOperator(partition), recordDescProvider
                         .getInputRecordDescriptor(this.odId, 0));
@@ -408,7 +408,7 @@
         }
     }
 
-    private IOperatorNodePushable createSelfReadingMapper(final IHyracksStageletContext ctx, IOperatorEnvironment env,
+    private IOperatorNodePushable createSelfReadingMapper(final IHyracksTaskContext ctx,
             final RecordDescriptor recordDescriptor, final int partition) {
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
             @Override
diff --git a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
index 915d09b..52f0c8a 100644
--- a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
+++ b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
@@ -32,7 +32,7 @@
 import org.apache.hadoop.util.ReflectionUtils;
 
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -143,7 +143,7 @@
 
     @SuppressWarnings("deprecation")
     @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IOperatorEnvironment env,
             final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
             throws HyracksDataException {
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
diff --git a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java
index 5857f36..0af0596 100644
--- a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java
+++ b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java
@@ -35,7 +35,7 @@
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.ReflectionUtils;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IDataReader;
 import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -348,7 +348,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         try {
             if (this.comparatorFactory == null) {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/AvgAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/AvgAggregatorDescriptorFactory.java
index c4d623c..45d49e2 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/AvgAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/AvgAggregatorDescriptorFactory.java
@@ -18,7 +18,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -39,7 +39,7 @@
     }
 
     @Override
-    public IAggregatorDescriptor createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDescriptor,
+    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
             RecordDescriptor outRecordDescriptor, int[] keyFields) throws HyracksDataException {
 
         if (this.outField < 0)
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ConcatAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ConcatAggregatorDescriptorFactory.java
index 90edf4a..1cc3340 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ConcatAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ConcatAggregatorDescriptorFactory.java
@@ -21,7 +21,7 @@
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -61,7 +61,7 @@
      * onto the output frame for indexing the aggregation result from the buffer.
      */
     @Override
-    public IAggregatorDescriptor createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDescriptor,
+    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
             RecordDescriptor outRecordDescriptor, final int[] keyFields) throws HyracksDataException {
 
         if (this.outField < 0)
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorDescriptorFactory.java
index c5f0a42..74ac53a 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorDescriptorFactory.java
@@ -18,7 +18,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -36,7 +36,7 @@
     }
 
     @Override
-    public IAggregatorDescriptor createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDescriptor,
+    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
             RecordDescriptor outRecordDescriptor, final int[] keyFields) throws HyracksDataException {
 
         if (this.outField < 0) {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptorFactory.java
index e324e1c..7f6928b 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptorFactory.java
@@ -16,7 +16,7 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
@@ -32,6 +32,6 @@
      * @return
      * @throws HyracksDataException
      */
-    IAggregatorDescriptor createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDescriptor,
+    IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
             RecordDescriptor outRecordDescriptor, int[] keyFields) throws HyracksDataException;
 }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IntSumAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IntSumAggregatorDescriptorFactory.java
index e07b123..770d36b 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IntSumAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IntSumAggregatorDescriptorFactory.java
@@ -18,7 +18,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -39,7 +39,7 @@
     }
 
     @Override
-    public IAggregatorDescriptor createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDescriptor,
+    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
             RecordDescriptor outRecordDescriptor, final int[] keyFields) throws HyracksDataException {
 
         if (this.outField < 0) {
@@ -111,7 +111,7 @@
 
             @Override
             public void reset() {
-                
+
             }
         };
     }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorDescriptorFactory.java
index 58d0da7..053b9e2 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorDescriptorFactory.java
@@ -15,7 +15,7 @@
 package edu.uci.ics.hyracks.dataflow.std.aggregators;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -29,7 +29,7 @@
     }
 
     @Override
-    public IAggregatorDescriptor createAggregator(final IHyracksStageletContext ctx,
+    public IAggregatorDescriptor createAggregator(final IHyracksTaskContext ctx,
             final RecordDescriptor inRecordDescriptor, final RecordDescriptor outRecordDescriptor, final int[] keyFields)
             throws HyracksDataException {
 
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java
index eb9c12d..8237110 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java
@@ -17,7 +17,7 @@
 import java.io.DataOutput;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -35,7 +35,7 @@
     }
 
     @Override
-    public IAccumulatingAggregator createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDesc,
+    public IAccumulatingAggregator createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
             RecordDescriptor outRecordDescriptor) throws HyracksDataException {
         final IFieldValueResultingAggregator aggregators[] = new IFieldValueResultingAggregator[aFactories.length];
         for (int i = 0; i < aFactories.length; ++i) {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java
index 4976230..11a02b3 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java
@@ -14,22 +14,20 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.base;
 
-import java.util.UUID;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
 
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
-
-public abstract class AbstractActivityNode implements IActivityNode {
+public abstract class AbstractActivityNode implements IActivity {
     private static final long serialVersionUID = 1L;
 
-    protected final ActivityNodeId id;
+    protected final ActivityId id;
 
-    public AbstractActivityNode() {
-        this.id = new ActivityNodeId(getOwner().getOperatorId(), UUID.randomUUID());
+    public AbstractActivityNode(ActivityId id) {
+        this.id = id;
     }
 
     @Override
-    public ActivityNodeId getActivityNodeId() {
+    public ActivityId getActivityId() {
         return id;
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
index e4613dd..f3535f8 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
@@ -14,16 +14,14 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.base;
 
-import java.util.UUID;
-
 import org.json.JSONException;
 import org.json.JSONObject;
 
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
 import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 public abstract class AbstractConnectorDescriptor implements IConnectorDescriptor {
@@ -31,7 +29,7 @@
     protected final ConnectorDescriptorId id;
 
     public AbstractConnectorDescriptor(JobSpecification spec) {
-        this.id = new ConnectorDescriptorId(UUID.randomUUID());
+        this.id = spec.createConnectorDescriptor();
         spec.getConnectorMap().put(id, this);
     }
 
@@ -44,14 +42,14 @@
         JSONObject jconn = new JSONObject();
 
         jconn.put("type", "connector");
-        jconn.put("id", getConnectorId().getId().toString());
+        jconn.put("id", getConnectorId().getId());
         jconn.put("java-class", getClass().getName());
 
         return jconn;
     }
 
     @Override
-    public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan,
+    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
             ICCApplicationContext appCtx) {
         // do nothing
     }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
new file mode 100644
index 0000000..aa550ae
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import java.util.BitSet;
+
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public abstract class AbstractMToNConnectorDescriptor extends AbstractConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public AbstractMToNConnectorDescriptor(JobSpecification spec) {
+        super(spec);
+    }
+
+    @Override
+    public void indicateTargetPartitions(int nProducerPartitions, int nConsumerPartitions, int producerIndex,
+            BitSet targetBitmap) {
+        targetBitmap.clear();
+        targetBitmap.set(0, nConsumerPartitions);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
index 4453412..15fc75d 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
@@ -14,17 +14,15 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.base;
 
-import java.util.UUID;
-
 import org.json.JSONException;
 import org.json.JSONObject;
 
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 public abstract class AbstractOperatorDescriptor implements IOperatorDescriptor {
@@ -41,7 +39,7 @@
     protected final int outputArity;
 
     public AbstractOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity) {
-        odId = new OperatorDescriptorId(UUID.randomUUID());
+        odId = spec.createOperatorDescriptorId();
         this.inputArity = inputArity;
         this.outputArity = outputArity;
         recordDescriptors = new RecordDescriptor[outputArity];
@@ -69,7 +67,7 @@
     }
 
     @Override
-    public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan,
+    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
             ICCApplicationContext appCtx) {
         // do nothing
     }
@@ -78,7 +76,7 @@
     public JSONObject toJSON() throws JSONException {
         JSONObject jop = new JSONObject();
         jop.put("type", "operator");
-        jop.put("id", getOperatorId().getId().toString());
+        jop.put("id", getOperatorId().getId());
         jop.put("java-class", getClass().getName());
         jop.put("in-arity", getInputArity());
         jop.put("out-arity", getOutputArity());
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
index 018b7b7..c4c3326 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
@@ -14,38 +14,29 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.base;
 
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.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 {
+public abstract class AbstractSingleActivityOperatorDescriptor extends AbstractOperatorDescriptor implements IActivity {
     private static final long serialVersionUID = 1L;
 
-    protected final ActivityNodeId activityNodeId;
+    protected final ActivityId activityNodeId;
 
     public AbstractSingleActivityOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity) {
         super(spec, inputArity, outputArity);
-        activityNodeId = new ActivityNodeId(odId, UUID.randomUUID());
+        activityNodeId = new ActivityId(odId, 0);
     }
 
     @Override
-    public ActivityNodeId getActivityNodeId() {
+    public ActivityId getActivityId() {
         return activityNodeId;
     }
 
     @Override
-    public final IOperatorDescriptor getOwner() {
-        return this;
-    }
-
-    @Override
-    public final void contributeTaskGraph(IActivityGraphBuilder builder) {
-        builder.addTask(this);
+    public final void contributeActivities(IActivityGraphBuilder builder) {
+        builder.addActivity(this);
         for (int i = 0; i < getInputArity(); ++i) {
             builder.addSourceEdge(i, this, i);
         }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/AbstractPartitionCollector.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/AbstractPartitionCollector.java
new file mode 100644
index 0000000..b2dad8f
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/AbstractPartitionCollector.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.collectors;
+
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public abstract class AbstractPartitionCollector implements IPartitionCollector {
+    protected final IHyracksTaskContext ctx;
+
+    protected final ConnectorDescriptorId connectorId;
+
+    protected final int receiverIndex;
+
+    public AbstractPartitionCollector(IHyracksTaskContext ctx, ConnectorDescriptorId connectorId, int receiverIndex) {
+        this.ctx = ctx;
+        this.connectorId = connectorId;
+        this.receiverIndex = receiverIndex;
+    }
+
+    @Override
+    public JobId getJobId() {
+        return ctx.getJobletContext().getJobId();
+    }
+
+    @Override
+    public ConnectorDescriptorId getConnectorId() {
+        return connectorId;
+    }
+
+    @Override
+    public int getReceiverIndex() {
+        return receiverIndex;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
new file mode 100644
index 0000000..5ff528e
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
@@ -0,0 +1,197 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.collectors;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+
+public class NonDeterministicPartitionCollector extends AbstractPartitionCollector {
+    private final FrameReader reader;
+
+    private final BitSet expectedPartitions;
+
+    private final int nSenderPartitions;
+
+    private final IInputChannel[] channels;
+
+    private final BitSet frameAvailability;
+
+    private final int[] availableFrameCounts;
+
+    private final BitSet eosSenders;
+
+    private BitSet closedSenders;
+
+    private int lastReadSender;
+
+    public NonDeterministicPartitionCollector(IHyracksTaskContext ctx, ConnectorDescriptorId connectorId,
+            int receiverIndex, int nSenderPartitions, BitSet expectedPartitions) {
+        super(ctx, connectorId, receiverIndex);
+        this.expectedPartitions = expectedPartitions;
+        this.nSenderPartitions = nSenderPartitions;
+        reader = new FrameReader();
+        channels = new IInputChannel[nSenderPartitions];
+        eosSenders = new BitSet(nSenderPartitions);
+        closedSenders = new BitSet(nSenderPartitions);
+        closedSenders.or(expectedPartitions);
+        closedSenders.flip(0, nSenderPartitions);
+        frameAvailability = new BitSet(nSenderPartitions);
+        availableFrameCounts = new int[nSenderPartitions];
+    }
+
+    @Override
+    public void open() throws HyracksException {
+        lastReadSender = 0;
+    }
+
+    @Override
+    public void addPartitions(Collection<PartitionChannel> partitions) throws HyracksException {
+        for (PartitionChannel pc : partitions) {
+            PartitionId pid = pc.getPartitionId();
+            IInputChannel channel = pc.getInputChannel();
+            channel.setAttachment(pid);
+            channel.registerMonitor(reader);
+            synchronized (this) {
+                channels[pid.getSenderIndex()] = channel;
+            }
+            channel.open();
+        }
+    }
+
+    @Override
+    public IFrameReader getReader() throws HyracksException {
+        return reader;
+    }
+
+    @Override
+    public void close() throws HyracksException {
+    }
+
+    private final class FrameReader implements IFrameReader, IInputChannelMonitor {
+        @Override
+        public void open() throws HyracksDataException {
+        }
+
+        @Override
+        public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            findNextSender();
+            if (lastReadSender >= 0) {
+                ByteBuffer srcFrame = channels[lastReadSender].getNextBuffer();
+                FrameUtils.copy(srcFrame, buffer);
+                channels[lastReadSender].recycleBuffer(srcFrame);
+                return true;
+            }
+            return false;
+        }
+
+        private void findNextSender() throws HyracksDataException {
+            synchronized (NonDeterministicPartitionCollector.this) {
+                while (true) {
+                    switch (lastReadSender) {
+                        default:
+                            lastReadSender = frameAvailability.nextSetBit(lastReadSender + 1);
+                            if (lastReadSender >= 0) {
+                                break;
+                            }
+                        case 0:
+                            lastReadSender = frameAvailability.nextSetBit(0);
+                    }
+                    if (lastReadSender >= 0) {
+                        assert availableFrameCounts[lastReadSender] > 0;
+                        if (--availableFrameCounts[lastReadSender] == 0) {
+                            frameAvailability.clear(lastReadSender);
+                        }
+                        return;
+                    }
+                    for (int i = eosSenders.nextSetBit(0); i >= 0; i = eosSenders.nextSetBit(i)) {
+                        channels[i].close();
+                        eosSenders.clear(i);
+                        closedSenders.set(i);
+                    }
+                    int nextClosedBitIndex = closedSenders.nextClearBit(0);
+                    if (nextClosedBitIndex < 0 || nextClosedBitIndex >= nSenderPartitions) {
+                        lastReadSender = -1;
+                        return;
+                    }
+                    try {
+                        NonDeterministicPartitionCollector.this.wait();
+                    } catch (InterruptedException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+            }
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            synchronized (NonDeterministicPartitionCollector.this) {
+                for (int i = closedSenders.nextClearBit(0); i >= 0 && i < nSenderPartitions; i = closedSenders
+                        .nextClearBit(i + 1)) {
+                    if (channels[i] != null) {
+                        channels[i].close();
+                        channels[i] = null;
+                    }
+                }
+            }
+        }
+
+        @Override
+        public void notifyDataAvailability(IInputChannel channel, int nFrames) {
+            synchronized (NonDeterministicPartitionCollector.this) {
+                PartitionId pid = (PartitionId) channel.getAttachment();
+                int senderIndex = pid.getSenderIndex();
+                availableFrameCounts[senderIndex] += nFrames;
+                frameAvailability.set(senderIndex);
+                NonDeterministicPartitionCollector.this.notifyAll();
+            }
+        }
+
+        @Override
+        public void notifyEndOfStream(IInputChannel channel) {
+            synchronized (NonDeterministicPartitionCollector.this) {
+                PartitionId pid = (PartitionId) channel.getAttachment();
+                int senderIndex = pid.getSenderIndex();
+                eosSenders.set(senderIndex);
+                NonDeterministicPartitionCollector.this.notifyAll();
+            }
+        }
+    }
+
+    @Override
+    public Collection<PartitionId> getRequiredPartitionIds() throws HyracksException {
+        Collection<PartitionId> c = new ArrayList<PartitionId>(expectedPartitions.cardinality());
+        for (int i = expectedPartitions.nextSetBit(0); i >= 0; i = expectedPartitions.nextSetBit(i + 1)) {
+            c.add(new PartitionId(getJobId(), getConnectorId(), i, getReceiverIndex()));
+        }
+        return c;
+    }
+
+    @Override
+    public void abort() {
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergePartitionCollector.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergePartitionCollector.java
new file mode 100644
index 0000000..8faee7b
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergePartitionCollector.java
@@ -0,0 +1,263 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.collectors;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunMergingFrameReader;
+
+public class SortMergePartitionCollector extends AbstractPartitionCollector {
+    private final int[] sortFields;
+
+    private final IBinaryComparator[] comparators;
+
+    private final RecordDescriptor recordDescriptor;
+
+    private final int maxConcurrentMerges;
+
+    private final IInputChannel[] channels;
+
+    private final int nSenders;
+
+    private final boolean stable;
+
+    private final FrameReader frameReader;
+
+    private final PartitionBatchManager pbm;
+
+    public SortMergePartitionCollector(IHyracksTaskContext ctx, ConnectorDescriptorId connectorId, int receiverIndex,
+            int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDescriptor,
+            int maxConcurrentMerges, int nSenders, boolean stable) {
+        super(ctx, connectorId, receiverIndex);
+        this.sortFields = sortFields;
+        this.comparators = comparators;
+        this.recordDescriptor = recordDescriptor;
+        this.maxConcurrentMerges = maxConcurrentMerges;
+        channels = new IInputChannel[nSenders];
+        this.nSenders = nSenders;
+        this.stable = stable;
+        this.frameReader = new FrameReader();
+        pbm = new NonDeterministicPartitionBatchManager();
+    }
+
+    @Override
+    public void open() throws HyracksException {
+    }
+
+    @Override
+    public void addPartitions(Collection<PartitionChannel> partitions) throws HyracksException {
+        for (PartitionChannel pc : partitions) {
+            PartitionId pid = pc.getPartitionId();
+            IInputChannel channel = pc.getInputChannel();
+            InputChannelFrameReader channelReader = new InputChannelFrameReader(channel);
+            channel.registerMonitor(channelReader);
+            channel.setAttachment(channelReader);
+            int senderIndex = pid.getSenderIndex();
+            synchronized (this) {
+                channels[senderIndex] = channel;
+            }
+            pbm.addPartition(senderIndex);
+            channel.open();
+        }
+    }
+
+    @Override
+    public IFrameReader getReader() throws HyracksException {
+        return frameReader;
+    }
+
+    @Override
+    public void close() throws HyracksException {
+
+    }
+
+    @Override
+    public Collection<PartitionId> getRequiredPartitionIds() throws HyracksException {
+        Collection<PartitionId> requiredPartitionIds = new ArrayList<PartitionId>();
+        for (int i = 0; i < nSenders; ++i) {
+            requiredPartitionIds.add(new PartitionId(getJobId(), getConnectorId(), i, receiverIndex));
+        }
+        return requiredPartitionIds;
+    }
+
+    @Override
+    public void abort() {
+
+    }
+
+    private abstract class PartitionBatchManager {
+        protected abstract void addPartition(int index);
+
+        protected abstract void getNextBatch(List<IFrameReader> batch, int size) throws HyracksDataException;
+    }
+
+    private class NonDeterministicPartitionBatchManager extends PartitionBatchManager {
+        private List<IFrameReader> partitions;
+
+        private List<IFrameReader> batch;
+
+        private int requiredSize;
+
+        public NonDeterministicPartitionBatchManager() {
+            partitions = new ArrayList<IFrameReader>();
+        }
+
+        @Override
+        protected void addPartition(int index) {
+            synchronized (SortMergePartitionCollector.this) {
+                if (batch != null && batch.size() < requiredSize) {
+                    batch.add((IFrameReader) channels[index].getAttachment());
+                    if (batch.size() == requiredSize) {
+                        SortMergePartitionCollector.this.notifyAll();
+                    }
+                } else {
+                    partitions.add((IFrameReader) channels[index].getAttachment());
+                }
+            }
+        }
+
+        @Override
+        protected void getNextBatch(List<IFrameReader> batch, int size) throws HyracksDataException {
+            synchronized (SortMergePartitionCollector.this) {
+                if (partitions.size() <= size) {
+                    batch.addAll(partitions);
+                    partitions.clear();
+                } else if (partitions.size() > size) {
+                    List<IFrameReader> sublist = partitions.subList(0, size);
+                    batch.addAll(sublist);
+                    sublist.clear();
+                }
+                if (batch.size() == size) {
+                    return;
+                }
+                this.batch = batch;
+                this.requiredSize = size;
+                while (batch.size() < size) {
+                    try {
+                        SortMergePartitionCollector.this.wait();
+                    } catch (InterruptedException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+                this.batch = null;
+            }
+        }
+    }
+
+    private static class InputChannelFrameReader implements IFrameReader, IInputChannelMonitor {
+        private final IInputChannel channel;
+
+        private int availableFrames;
+
+        private boolean eos;
+
+        public InputChannelFrameReader(IInputChannel channel) {
+            this.channel = channel;
+            availableFrames = 0;
+            eos = false;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+        }
+
+        @Override
+        public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            synchronized (this) {
+                while (!eos && availableFrames <= 0) {
+                    try {
+                        wait();
+                    } catch (InterruptedException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+                if (availableFrames <=0 && eos) {
+                    return false;
+                }
+                --availableFrames;
+            }
+            ByteBuffer srcBuffer = channel.getNextBuffer();
+            FrameUtils.copy(srcBuffer, buffer);
+            channel.recycleBuffer(srcBuffer);
+            return true;
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+
+        }
+
+        @Override
+        public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
+            availableFrames += nFrames;
+            notifyAll();
+        }
+
+        @Override
+        public synchronized void notifyEndOfStream(IInputChannel channel) {
+            eos = true;
+            notifyAll();
+        }
+    }
+
+    private class FrameReader implements IFrameReader {
+        private RunMergingFrameReader merger;
+
+        @Override
+        public void open() throws HyracksDataException {
+            if (maxConcurrentMerges >= nSenders) {
+                List<ByteBuffer> inFrames = new ArrayList<ByteBuffer>();
+                for (int i = 0; i < nSenders; ++i) {
+                    inFrames.add(ByteBuffer.allocate(ctx.getFrameSize()));
+                }
+                List<IFrameReader> batch = new ArrayList<IFrameReader>();
+                pbm.getNextBatch(batch, nSenders);
+                merger = new RunMergingFrameReader(ctx, batch.toArray(new IFrameReader[nSenders]), inFrames,
+                        sortFields, comparators, recordDescriptor);
+            } else {
+                // multi level merge.
+                throw new HyracksDataException("Not yet supported");
+            }
+            merger.open();
+        }
+
+        @Override
+        public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            buffer.position(buffer.capacity());
+            buffer.limit(buffer.capacity());
+            return merger.nextFrame(buffer);
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            merger.close();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningConnectorDescriptor.java
deleted file mode 100644
index b814867..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningConnectorDescriptor.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.connectors;
-
-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.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
-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.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.NonDeterministicFrameReader;
-import edu.uci.ics.hyracks.dataflow.std.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(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
-            IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
-            throws HyracksDataException {
-        final HashDataWriter hashWriter = new HashDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc,
-                tpcf.createPartitioner());
-        return hashWriter;
-    }
-
-    @Override
-    public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
-            IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
-            throws HyracksDataException {
-        return new NonDeterministicFrameReader(ctx, demux);
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningMergingConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningMergingConnectorDescriptor.java
deleted file mode 100644
index 36f8d5b..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningMergingConnectorDescriptor.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.connectors;
-
-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.context.IHyracksStageletContext;
-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.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.SortMergeFrameReader;
-import edu.uci.ics.hyracks.dataflow.std.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(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
-            IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
-            throws HyracksDataException {
-        final HashDataWriter hashWriter = new HashDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc,
-                tpcf.createPartitioner());
-        return hashWriter;
-    }
-
-    @Override
-    public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
-            IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
-            throws HyracksDataException {
-        IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
-        for (int i = 0; i < comparatorFactories.length; ++i) {
-            comparators[i] = comparatorFactories[i].createBinaryComparator();
-        }
-        return new SortMergeFrameReader(ctx, demux, sortFields, comparators, recordDesc);
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
new file mode 100644
index 0000000..12935af
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.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.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicPartitionCollector;
+
+public class MToNPartitioningConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private ITuplePartitionComputerFactory tpcf;
+
+    public MToNPartitioningConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf) {
+        super(spec);
+        this.tpcf = tpcf;
+    }
+
+    @Override
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        final PartitionDataWriter hashWriter = new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory,
+                recordDesc, tpcf.createPartitioner());
+        return hashWriter;
+    }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        BitSet expectedPartitions = new BitSet(nProducerPartitions);
+        expectedPartitions.set(0, nProducerPartitions);
+        return new NonDeterministicPartitionCollector(ctx, getConnectorId(), index, nProducerPartitions,
+                expectedPartitions);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
new file mode 100644
index 0000000..54979f0
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.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.dataflow.std.connectors;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.SortMergePartitionCollector;
+
+public class MToNPartitioningMergingConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final ITuplePartitionComputerFactory tpcf;
+    private final int[] sortFields;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final boolean stable;
+
+    public MToNPartitioningMergingConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf,
+            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories) {
+        this(spec, tpcf, sortFields, comparatorFactories, false);
+    }
+
+    public MToNPartitioningMergingConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf,
+            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories, boolean stable) {
+        super(spec);
+        this.tpcf = tpcf;
+        this.sortFields = sortFields;
+        this.comparatorFactories = comparatorFactories;
+        this.stable = stable;
+    }
+
+    @Override
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        final PartitionDataWriter hashWriter = new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory,
+                recordDesc, tpcf.createPartitioner());
+        return hashWriter;
+    }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        return new SortMergePartitionCollector(ctx, getConnectorId(), index, sortFields, comparators, recordDesc,
+                nProducerPartitions, nProducerPartitions, stable);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
deleted file mode 100644
index e108396..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.connectors;
-
-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.context.IHyracksStageletContext;
-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.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.NonDeterministicFrameReader;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
-
-public class MToNRangePartitioningConnectorDescriptor extends AbstractConnectorDescriptor {
-    private static final long serialVersionUID = 1L;
-
-    private class RangeDataWriter implements IFrameWriter {
-        private final IFrameWriter[] epWriters;
-        private final FrameTupleAppender[] appenders;
-        private final FrameTupleAccessor tupleAccessor;
-
-        public RangeDataWriter(IHyracksStageletContext ctx, int consumerPartitionCount, IFrameWriter[] epWriters,
-                FrameTupleAppender[] appenders, RecordDescriptor recordDescriptor) {
-            this.epWriters = epWriters;
-            this.appenders = appenders;
-            tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), 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);
-            }
-        }
-
-        @Override
-        public void flush() throws HyracksDataException {
-            for (int i = 0; i < appenders.length; ++i) {
-                FrameTupleAppender appender = appenders[i];
-                if (appender.getTupleCount() > 0) {
-                    ByteBuffer buffer = appender.getBuffer();
-                    IFrameWriter frameWriter = epWriters[i];
-                    flushFrame(buffer, frameWriter);
-                    epWriters[i].flush();
-                    appender.reset(buffer, 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(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
-            IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
-            throws HyracksDataException {
-        final IFrameWriter[] epWriters = new IFrameWriter[nConsumerPartitions];
-        final FrameTupleAppender[] appenders = new FrameTupleAppender[nConsumerPartitions];
-        for (int i = 0; i < nConsumerPartitions; ++i) {
-            try {
-                epWriters[i] = edwFactory.createFrameWriter(i);
-                appenders[i] = new FrameTupleAppender(ctx.getFrameSize());
-                appenders[i].reset(ctx.allocateFrame(), true);
-            } catch (IOException e) {
-                throw new HyracksDataException(e);
-            }
-        }
-        final RangeDataWriter rangeWriter = new RangeDataWriter(ctx, nConsumerPartitions, epWriters, appenders,
-                recordDesc);
-        return rangeWriter;
-    }
-
-    @Override
-    public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
-            IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
-            throws HyracksDataException {
-        return new NonDeterministicFrameReader(ctx, demux);
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
index 37f68b7..5297a99 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
@@ -15,19 +15,19 @@
 package edu.uci.ics.hyracks.dataflow.std.connectors;
 
 import java.nio.ByteBuffer;
+import java.util.BitSet;
 
-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.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.NonDeterministicFrameReader;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicPartitionCollector;
 
-public class MToNReplicatingConnectorDescriptor extends AbstractConnectorDescriptor {
+public class MToNReplicatingConnectorDescriptor extends AbstractMToNConnectorDescriptor {
     public MToNReplicatingConnectorDescriptor(JobSpecification spec) {
         super(spec);
     }
@@ -35,8 +35,8 @@
     private static final long serialVersionUID = 1L;
 
     @Override
-    public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
-            IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         final IFrameWriter[] epWriters = new IFrameWriter[nConsumerPartitions];
         for (int i = 0; i < nConsumerPartitions; ++i) {
@@ -78,9 +78,11 @@
     }
 
     @Override
-    public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
-            IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
-            throws HyracksDataException {
-        return new NonDeterministicFrameReader(ctx, demux);
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        BitSet expectedPartitions = new BitSet(nProducerPartitions);
+        expectedPartitions.set(0, nProducerPartitions);
+        return new NonDeterministicPartitionCollector(ctx, getConnectorId(), index, nProducerPartitions,
+                expectedPartitions);
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
index 6ee9ab37..67130c9 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
@@ -14,22 +14,23 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.connectors;
 
+import java.util.BitSet;
+
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
 import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.RelationalExpression;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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.JobActivityGraph;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.NonDeterministicFrameReader;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicPartitionCollector;
 
 public class OneToOneConnectorDescriptor extends AbstractConnectorDescriptor {
     private static final long serialVersionUID = 1L;
@@ -39,28 +40,36 @@
     }
 
     @Override
-    public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
-            IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         return edwFactory.createFrameWriter(index);
     }
 
     @Override
-    public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
-            IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
-            throws HyracksDataException {
-        return new NonDeterministicFrameReader(ctx, demux);
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        BitSet expectedPartitions = new BitSet(nProducerPartitions);
+        expectedPartitions.set(index);
+        return new NonDeterministicPartitionCollector(ctx, getConnectorId(), index, nProducerPartitions,
+                expectedPartitions);
     }
 
     @Override
-    public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan,
+    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
             ICCApplicationContext appCtx) {
         JobSpecification jobSpec = plan.getJobSpecification();
         IOperatorDescriptor consumer = jobSpec.getConsumer(this);
         IOperatorDescriptor producer = jobSpec.getProducer(this);
 
-        constraintAcceptor.addConstraintExpression(new RelationalExpression(new PartitionCountExpression(consumer
-                .getOperatorId()), new PartitionCountExpression(producer.getOperatorId()),
-                RelationalExpression.Operator.EQUAL));
+        constraintAcceptor.addConstraint(new Constraint(new PartitionCountExpression(consumer.getOperatorId()),
+                new PartitionCountExpression(producer.getOperatorId())));
+    }
+
+    @Override
+    public void indicateTargetPartitions(int nProducerPartitions, int nConsumerPartitions, int producerIndex,
+            BitSet targetBitmap) {
+        targetBitmap.clear();
+        targetBitmap.set(producerIndex);
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashDataWriter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
similarity index 78%
rename from hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashDataWriter.java
rename to hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
index 44dd84c..7054535 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashDataWriter.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
@@ -18,30 +18,29 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 
-public class HashDataWriter implements IFrameWriter {
+public class PartitionDataWriter implements IFrameWriter {
     private final int consumerPartitionCount;
-    private final IFrameWriter[] epWriters;
+    private final IFrameWriter[] pWriters;
     private final FrameTupleAppender[] appenders;
     private final FrameTupleAccessor tupleAccessor;
     private final ITuplePartitionComputer tpc;
 
-    public HashDataWriter(IHyracksStageletContext ctx, int consumerPartitionCount,
-            IEndpointDataWriterFactory edwFactory, RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc)
-            throws HyracksDataException {
+    public PartitionDataWriter(IHyracksTaskContext ctx, int consumerPartitionCount, IPartitionWriterFactory pwFactory,
+            RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc) throws HyracksDataException {
         this.consumerPartitionCount = consumerPartitionCount;
-        epWriters = new IFrameWriter[consumerPartitionCount];
+        pWriters = new IFrameWriter[consumerPartitionCount];
         appenders = new FrameTupleAppender[consumerPartitionCount];
         for (int i = 0; i < consumerPartitionCount; ++i) {
             try {
-                epWriters[i] = edwFactory.createFrameWriter(i);
+                pWriters[i] = pwFactory.createFrameWriter(i);
                 appenders[i] = new FrameTupleAppender(ctx.getFrameSize());
                 appenders[i].reset(ctx.allocateFrame(), true);
             } catch (IOException e) {
@@ -54,11 +53,11 @@
 
     @Override
     public void close() throws HyracksDataException {
-        for (int i = 0; i < epWriters.length; ++i) {
+        for (int i = 0; i < pWriters.length; ++i) {
             if (appenders[i].getTupleCount() > 0) {
-                flushFrame(appenders[i].getBuffer(), epWriters[i]);
+                flushFrame(appenders[i].getBuffer(), pWriters[i]);
             }
-            epWriters[i].close();
+            pWriters[i].close();
         }
     }
 
@@ -70,8 +69,8 @@
 
     @Override
     public void open() throws HyracksDataException {
-        for (int i = 0; i < epWriters.length; ++i) {
-            epWriters[i].open();
+        for (int i = 0; i < pWriters.length; ++i) {
+            pWriters[i].open();
             appenders[i].reset(appenders[i].getBuffer(), true);
         }
     }
@@ -85,7 +84,7 @@
             FrameTupleAppender appender = appenders[h];
             if (!appender.append(tupleAccessor, i)) {
                 ByteBuffer appenderBuffer = appender.getBuffer();
-                flushFrame(appenderBuffer, epWriters[h]);
+                flushFrame(appenderBuffer, pWriters[h]);
                 appender.reset(appenderBuffer, true);
                 if (!appender.append(tupleAccessor, i)) {
                     throw new IllegalStateException();
@@ -100,9 +99,9 @@
             FrameTupleAppender appender = appenders[i];
             if (appender.getTupleCount() > 0) {
                 ByteBuffer buffer = appender.getBuffer();
-                IFrameWriter frameWriter = epWriters[i];
+                IFrameWriter frameWriter = pWriters[i];
                 flushFrame(buffer, frameWriter);
-                epWriters[i].flush();
+                pWriters[i].flush();
                 appender.reset(buffer, true);
             }
         }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
index b2265dd..03438d3 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
@@ -16,7 +16,7 @@
 
 import java.io.File;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -100,7 +100,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new DeserializedOperatorNodePushable(ctx, new DeserializedFileScanOperator(partition), null);
     }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
index 8e4199b..ae1a722 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.file;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -85,7 +85,7 @@
     protected abstract IRecordWriter createRecordWriter(FileSplit fileSplit, int index) throws Exception;
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new DeserializedOperatorNodePushable(ctx, new FileWriteOperator(partition),
                 recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
index 954d64a..0096576 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
@@ -23,7 +23,7 @@
 import java.util.Arrays;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -42,7 +42,7 @@
     }
 
     @Override
-    public ITupleParser createTupleParser(final IHyracksStageletContext ctx) {
+    public ITupleParser createTupleParser(final IHyracksTaskContext ctx) {
         return new ITupleParser() {
             @Override
             public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
index 7cbb2ed..1d580b1 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
@@ -19,7 +19,7 @@
 import java.io.FileNotFoundException;
 import java.io.InputStream;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -45,7 +45,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         final FileSplit split = fileSplitProvider.getFileSplits()[partition];
         final ITupleParser tp = tupleParserFactory.createTupleParser(ctx);
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
index 1628984..21e197d 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
@@ -20,7 +20,7 @@
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -40,7 +40,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
         final FileSplit[] splits = fileSplitProvider.getFileSplits();
         return new AbstractUnaryInputSinkOperatorNodePushable() {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
index a11dae0..eb051f5 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
@@ -16,8 +16,8 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 public interface ITupleParserFactory extends Serializable {
-    public ITupleParser createTupleParser(IHyracksStageletContext ctx);
+    public ITupleParser createTupleParser(IHyracksTaskContext ctx);
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
index d599fee..cd320dd1 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
@@ -20,7 +20,7 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -61,7 +61,7 @@
      * @see edu.uci.ics.hyracks.api.dataflow.IActivityNode#createPushRuntime(edu.uci.ics.hyracks.api.context.IHyracksContext, edu.uci.ics.hyracks.api.job.IOperatorEnvironment, edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider, int, int)
      */
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
             throws HyracksDataException {
         // Output files
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
index 65f7f08..73de7f5 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
@@ -23,9 +23,9 @@
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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;
@@ -101,32 +101,30 @@
     }
 
     @Override
-    public void contributeTaskGraph(IActivityGraphBuilder builder) {
-        AggregateActivity aggregateAct = new AggregateActivity();
-        MergeActivity mergeAct = new MergeActivity();
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        AggregateActivity aggregateAct = new AggregateActivity(new ActivityId(getOperatorId(), 0));
+        MergeActivity mergeAct = new MergeActivity(new ActivityId(odId, 1));
 
-        builder.addTask(aggregateAct);
+        builder.addActivity(aggregateAct);
         builder.addSourceEdge(0, aggregateAct, 0);
 
-        builder.addTask(mergeAct);
+        builder.addActivity(mergeAct);
         builder.addTargetEdge(0, mergeAct, 0);
 
         builder.addBlockingEdge(aggregateAct, mergeAct);
     }
 
     private class AggregateActivity extends AbstractActivityNode {
-
         private static final long serialVersionUID = 1L;
 
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return ExternalGroupOperatorDescriptor.this;
+        public AggregateActivity(ActivityId id) {
+            super(id);
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) throws HyracksDataException {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+                throws HyracksDataException {
             final ISpillableTable gTable = spillableTableFactory.buildSpillableTable(ctx, keyFields,
                     comparatorFactories, firstNormalizerFactory, aggregatorFactory,
                     recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0), recordDescriptors[0],
@@ -191,7 +189,7 @@
                 private void flushFramesToRun() throws HyracksDataException {
                     FileReference runFile;
                     try {
-                        runFile = ctx.getJobletContext().createWorkspaceFile(
+                        runFile = ctx.getJobletContext().createManagedWorkspaceFile(
                                 ExternalGroupOperatorDescriptor.class.getSimpleName());
                     } catch (IOException e) {
                         throw new HyracksDataException(e);
@@ -215,18 +213,16 @@
     }
 
     private class MergeActivity extends AbstractActivityNode {
-
         private static final long serialVersionUID = 1L;
 
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return ExternalGroupOperatorDescriptor.this;
+        public MergeActivity(ActivityId id) {
+            super(id);
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) throws HyracksDataException {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+                throws HyracksDataException {
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
             for (int i = 0; i < comparatorFactories.length; ++i) {
                 comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -325,7 +321,7 @@
                         runNumber = runs.size();
                     } else {
                         runNumber = framesLimit - 2;
-                        newRun = ctx.getJobletContext().createWorkspaceFile(
+                        newRun = ctx.getJobletContext().createManagedWorkspaceFile(
                                 ExternalGroupOperatorDescriptor.class.getSimpleName());
                         writer = new RunFileWriter(newRun, ctx.getIOManager());
                         writer.open();
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
index 07a5b81..5733c11 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
@@ -20,7 +20,7 @@
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -60,7 +60,7 @@
     }
 
     private static final int INIT_ACCUMULATORS_SIZE = 8;
-    private final IHyracksStageletContext ctx;
+    private final IHyracksTaskContext ctx;
     private final FrameTupleAppender appender;
     private final List<ByteBuffer> buffers;
     private final Link[] table;
@@ -79,7 +79,7 @@
 
     private final FrameTupleAccessor storedKeysAccessor;
 
-    GroupingHashTable(IHyracksStageletContext ctx, int[] fields, IBinaryComparatorFactory[] comparatorFactories,
+    GroupingHashTable(IHyracksTaskContext ctx, int[] fields, IBinaryComparatorFactory[] comparatorFactories,
             ITuplePartitionComputerFactory tpcf, IAccumulatingAggregatorFactory aggregatorFactory,
             RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor, int tableSize) {
         this.ctx = ctx;
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
index 50d467c..cf954f6 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
@@ -16,9 +16,9 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -57,12 +57,12 @@
     }
 
     @Override
-    public void contributeTaskGraph(IActivityGraphBuilder builder) {
-        HashBuildActivity ha = new HashBuildActivity();
-        builder.addTask(ha);
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        HashBuildActivity ha = new HashBuildActivity(new ActivityId(odId, 0));
+        builder.addActivity(ha);
 
-        OutputActivity oa = new OutputActivity();
-        builder.addTask(oa);
+        OutputActivity oa = new OutputActivity(new ActivityId(odId, 1));
+        builder.addActivity(oa);
 
         builder.addSourceEdge(0, ha, 0);
         builder.addTargetEdge(0, oa, 0);
@@ -72,10 +72,13 @@
     private class HashBuildActivity extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
+        public HashBuildActivity(ActivityId id) {
+            super(id);
+        }
+
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, final IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             final FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(),
                     recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
             return new AbstractUnaryInputSinkOperatorNodePushable() {
@@ -107,18 +110,17 @@
                 }
             };
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return HashGroupOperatorDescriptor.this;
-        }
     }
 
     private class OutputActivity extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
+        public OutputActivity(ActivityId id) {
+            super(id);
+        }
+
         @Override
-        public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, final IOperatorEnvironment env,
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, final IOperatorEnvironment env,
                 IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             return new AbstractUnaryOutputSourceOperatorNodePushable() {
                 @Override
@@ -131,10 +133,5 @@
                 }
             };
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return HashGroupOperatorDescriptor.this;
-        }
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java
index 3cc2fad..af5d16b 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java
@@ -19,7 +19,7 @@
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
@@ -52,7 +52,7 @@
     }
 
     @Override
-    public ISpillableTable buildSpillableTable(final IHyracksStageletContext ctx, final int[] keyFields,
+    public ISpillableTable buildSpillableTable(final IHyracksTaskContext ctx, final int[] keyFields,
             final IBinaryComparatorFactory[] comparatorFactories,
             final INormalizedKeyComputerFactory firstKeyNormalizerFactory,
             final IAggregatorDescriptorFactory aggregateDescriptorFactory, final RecordDescriptor inRecordDescriptor,
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
index 8eff82f..978f671 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
@@ -16,12 +16,11 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface IAccumulatingAggregatorFactory extends Serializable {
-    IAccumulatingAggregator createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDesc,
+    IAccumulatingAggregator createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
             RecordDescriptor outRecordDescriptor) throws HyracksDataException;
-
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java
index 07f0ac0..0ff1d1d 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java
@@ -16,7 +16,7 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -27,7 +27,7 @@
  * @author jarodwen
  */
 public interface ISpillableTableFactory extends Serializable {
-    ISpillableTable buildSpillableTable(IHyracksStageletContext ctx, int[] keyFields,
+    ISpillableTable buildSpillableTable(IHyracksTaskContext ctx, int[] keyFields,
             IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory normalizedKeyComputerFactory,
             IAggregatorDescriptorFactory aggregatorFactory, RecordDescriptor inRecordDescriptor,
             RecordDescriptor outRecordDescriptor, int framesLimit) throws HyracksDataException;
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
index ea86d19..b5568e6 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -48,7 +48,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
         for (int i = 0; i < comparatorFactories.length; ++i) {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
index 8817113..60a1dc3 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
@@ -17,7 +17,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -37,7 +37,7 @@
     private final FrameTupleAppender appender;
     private boolean first;
 
-    public PreclusteredGroupWriter(IHyracksStageletContext ctx, int[] groupFields, IBinaryComparator[] comparators,
+    public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields, IBinaryComparator[] comparators,
             IAccumulatingAggregator aggregator, RecordDescriptor inRecordDesc, IFrameWriter writer) {
         this.groupFields = groupFields;
         this.comparators = comparators;
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
index 1a69952..c60c0af 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
@@ -16,9 +16,9 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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;
@@ -97,20 +97,20 @@
     }
 
     @Override
-    public void contributeTaskGraph(IActivityGraphBuilder builder) {
-        HashPartitionActivityNode part0 = new HashPartitionActivityNode(RELATION0, keys0, 0);
-        HashPartitionActivityNode part1 = new HashPartitionActivityNode(RELATION1, keys1, 1);
-        JoinActivityNode join = new JoinActivityNode();
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        HashPartitionActivityNode rpart = new HashPartitionActivityNode(new ActivityId(odId, 0), RELATION0, keys0, 0);
+        HashPartitionActivityNode spart = new HashPartitionActivityNode(new ActivityId(odId, 1), RELATION1, keys1, 1);
+        JoinActivityNode join = new JoinActivityNode(new ActivityId(odId, 2));
 
-        builder.addTask(part0);
-        builder.addSourceEdge(0, part0, 0);
+        builder.addActivity(rpart);
+        builder.addSourceEdge(0, rpart, 0);
 
-        builder.addTask(part1);
-        builder.addSourceEdge(1, part1, 0);
+        builder.addActivity(spart);
+        builder.addSourceEdge(1, spart, 0);
 
-        builder.addTask(join);
-        builder.addBlockingEdge(part0, part1);
-        builder.addBlockingEdge(part1, join);
+        builder.addActivity(join);
+        builder.addBlockingEdge(rpart, spart);
+        builder.addBlockingEdge(spart, join);
 
         builder.addTargetEdge(0, join, 0);
     }
@@ -125,16 +125,16 @@
         private int operatorInputIndex;
         private int keys[];
 
-        public HashPartitionActivityNode(String partitionsKey, int keys[], int operatorInputIndex) {
+        public HashPartitionActivityNode(ActivityId id, String partitionsKey, int keys[], int operatorInputIndex) {
+            super(id);
             this.partitionsKey = partitionsKey;
             this.keys = keys;
             this.operatorInputIndex = operatorInputIndex;
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, final IRecordDescriptorProvider recordDescProvider, int partition,
-                final int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                final IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
             for (int i = 0; i < comparatorFactories.length; ++i) {
                 comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -175,7 +175,7 @@
                 private void write(int i, ByteBuffer head) throws HyracksDataException {
                     RunFileWriter writer = fWriters[i];
                     if (writer == null) {
-                        FileReference file = ctx.getJobletContext().createWorkspaceFile(partitionsKey);
+                        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(partitionsKey);
                         writer = new RunFileWriter(file, ctx.getIOManager());
                         writer.open();
                         fWriters[i] = writer;
@@ -221,20 +221,18 @@
             };
             return op;
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return GraceHashJoinOperatorDescriptor.this;
-        }
     }
 
     private class JoinActivityNode extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
+        public JoinActivityNode(ActivityId id) {
+            super(id);
+        }
+
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, final IRecordDescriptorProvider recordDescProvider, int partition,
-                final int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                final IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
             for (int i = 0; i < comparatorFactories.length; ++i) {
                 comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -314,10 +312,5 @@
             };
             return op;
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return GraceHashJoinOperatorDescriptor.this;
-        }
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index 516bb88..7c9134f 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -16,9 +16,9 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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;
@@ -116,14 +116,14 @@
     }
 
     @Override
-    public void contributeTaskGraph(IActivityGraphBuilder builder) {
-        BuildAndPartitionActivityNode phase1 = new BuildAndPartitionActivityNode(BUILDRELATION);
-        PartitionAndJoinActivityNode phase2 = new PartitionAndJoinActivityNode(PROBERELATION);
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        BuildAndPartitionActivityNode phase1 = new BuildAndPartitionActivityNode(new ActivityId(odId, 0), BUILDRELATION);
+        PartitionAndJoinActivityNode phase2 = new PartitionAndJoinActivityNode(new ActivityId(odId, 1), PROBERELATION);
 
-        builder.addTask(phase1);
+        builder.addActivity(phase1);
         builder.addSourceEdge(1, phase1, 0);
 
-        builder.addTask(phase2);
+        builder.addActivity(phase2);
         builder.addSourceEdge(0, phase2, 0);
 
         builder.addBlockingEdge(phase1, phase2);
@@ -135,16 +135,14 @@
         private static final long serialVersionUID = 1L;
         private String relationName;
 
-        public BuildAndPartitionActivityNode(String relationName) {
-            super();
+        public BuildAndPartitionActivityNode(ActivityId id, String relationName) {
+            super(id);
             this.relationName = relationName;
-
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                final int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
             final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -310,7 +308,7 @@
                 private void write(int i, ByteBuffer head) throws HyracksDataException {
                     RunFileWriter writer = fWriters[i];
                     if (writer == null) {
-                        FileReference file = ctx.getJobletContext().createWorkspaceFile(relationName);
+                        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(relationName);
                         writer = new RunFileWriter(file, ctx.getIOManager());
                         writer.open();
                         fWriters[i] = writer;
@@ -320,26 +318,20 @@
             };
             return op;
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return HybridHashJoinOperatorDescriptor.this;
-        }
     }
 
     private class PartitionAndJoinActivityNode extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
         private String relationName;
 
-        public PartitionAndJoinActivityNode(String relationName) {
-            super();
+        public PartitionAndJoinActivityNode(ActivityId id, String relationName) {
+            super(id);
             this.relationName = relationName;
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                final int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
             final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -524,7 +516,7 @@
                 private void write(int i, ByteBuffer head) throws HyracksDataException {
                     RunFileWriter writer = probeWriters[i];
                     if (writer == null) {
-                        FileReference file = ctx.createWorkspaceFile(relationName);
+                        FileReference file = ctx.createManagedWorkspaceFile(relationName);
                         writer = new RunFileWriter(file, ctx.getIOManager());
                         writer.open();
                         probeWriters[i] = writer;
@@ -534,10 +526,5 @@
             };
             return op;
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return HybridHashJoinOperatorDescriptor.this;
-        }
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
index d1d528c..fad12c1 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -21,7 +21,7 @@
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -43,7 +43,7 @@
     private final boolean isLeftOuter;
     private final ArrayTupleBuilder nullTupleBuild;
 
-    public InMemoryHashJoin(IHyracksStageletContext ctx, int tableSize, FrameTupleAccessor accessor0,
+    public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
             ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
             FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1)
             throws HyracksDataException {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
index 13af25d..ce8be1c 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -16,9 +16,9 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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;
@@ -82,14 +82,14 @@
     }
 
     @Override
-    public void contributeTaskGraph(IActivityGraphBuilder builder) {
-        HashBuildActivityNode hba = new HashBuildActivityNode();
-        HashProbeActivityNode hpa = new HashProbeActivityNode();
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        HashBuildActivityNode hba = new HashBuildActivityNode(new ActivityId(odId, 0));
+        HashProbeActivityNode hpa = new HashProbeActivityNode(new ActivityId(odId, 1));
 
-        builder.addTask(hba);
+        builder.addActivity(hba);
         builder.addSourceEdge(1, hba, 0);
 
-        builder.addTask(hpa);
+        builder.addActivity(hpa);
         builder.addSourceEdge(0, hpa, 0);
 
         builder.addTargetEdge(0, hpa, 0);
@@ -100,10 +100,13 @@
     private class HashBuildActivityNode extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
+        public HashBuildActivityNode(ActivityId id) {
+            super(id);
+        }
+
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -149,20 +152,18 @@
             };
             return op;
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return InMemoryHashJoinOperatorDescriptor.this;
-        }
     }
 
     private class HashProbeActivityNode extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
+        public HashProbeActivityNode(ActivityId id) {
+            super(id);
+        }
+
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
                 private InMemoryHashJoin joiner;
 
@@ -191,10 +192,5 @@
             };
             return op;
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return InMemoryHashJoinOperatorDescriptor.this;
-        }
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
index dd985d3..7e84229 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -19,7 +19,7 @@
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -38,12 +38,12 @@
     private final ByteBuffer innerBuffer;
     private final List<ByteBuffer> outBuffers;
     private final int memSize;
-    private final IHyracksStageletContext ctx;
+    private final IHyracksTaskContext ctx;
     private RunFileReader runFileReader;
     private int currentMemSize = 0;
     private final RunFileWriter runFileWriter;
 
-    public NestedLoopJoin(IHyracksStageletContext ctx, FrameTupleAccessor accessor0, FrameTupleAccessor accessor1,
+    public NestedLoopJoin(IHyracksTaskContext ctx, FrameTupleAccessor accessor0, FrameTupleAccessor accessor1,
             ITuplePairComparator comparators, int memSize) throws HyracksDataException {
         this.accessorInner = accessor1;
         this.accessorOuter = accessor0;
@@ -56,7 +56,7 @@
         this.memSize = memSize;
         this.ctx = ctx;
 
-        FileReference file = ctx.getJobletContext().createWorkspaceFile(
+        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
                 this.getClass().getSimpleName() + this.toString());
         runFileWriter = new RunFileWriter(file, ctx.getIOManager());
         runFileWriter.open();
@@ -163,4 +163,4 @@
         }
         return 0;
     }
-}
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
index efee8ff..9e69e4b 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
@@ -17,9 +17,9 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
@@ -51,14 +51,14 @@
     }
 
     @Override
-    public void contributeTaskGraph(IActivityGraphBuilder builder) {
-        JoinCacheActivityNode jc = new JoinCacheActivityNode();
-        NestedLoopJoinActivityNode nlj = new NestedLoopJoinActivityNode();
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        JoinCacheActivityNode jc = new JoinCacheActivityNode(new ActivityId(getOperatorId(), 0));
+        NestedLoopJoinActivityNode nlj = new NestedLoopJoinActivityNode(new ActivityId(getOperatorId(), 1));
 
-        builder.addTask(jc);
+        builder.addActivity(jc);
         builder.addSourceEdge(1, jc, 0);
 
-        builder.addTask(nlj);
+        builder.addActivity(nlj);
         builder.addSourceEdge(0, nlj, 0);
 
         builder.addTargetEdge(0, nlj, 0);
@@ -68,10 +68,13 @@
     private class JoinCacheActivityNode extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
+        public JoinCacheActivityNode(ActivityId id) {
+            super(id);
+        }
+
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
             final ITuplePairComparator comparator = comparatorFactory.createTuplePairComparator();
@@ -105,20 +108,18 @@
             };
             return op;
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return NestedLoopJoinOperatorDescriptor.this;
-        }
     }
 
     private class NestedLoopJoinActivityNode extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
+        public NestedLoopJoinActivityNode(ActivityId id) {
+            super(id);
+        }
+
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
 
             IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
                 private NestedLoopJoin joiner;
@@ -148,10 +149,5 @@
             };
             return op;
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return NestedLoopJoinOperatorDescriptor.this;
-        }
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
index 37b5165..a142c94 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.map;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -69,7 +69,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new DeserializedOperatorNodePushable(ctx, new MapperOperator(),
                 recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
index 4ec6393..48e53b5 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.misc;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -41,7 +41,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         return new ConstantTupleSourceOperatorNodePushable(ctx, fieldSlots, tupleData, tupleSize);
     }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
index 8784d4a..34a8e5e 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
@@ -17,20 +17,20 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
 
 public class ConstantTupleSourceOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
-    private IHyracksStageletContext ctx;
+    private IHyracksTaskContext ctx;
 
     private int[] fieldSlots;
     private byte[] tupleData;
     private int tupleSize;
 
-    public ConstantTupleSourceOperatorNodePushable(IHyracksStageletContext ctx, int[] fieldSlots, byte[] tupleData,
+    public ConstantTupleSourceOperatorNodePushable(IHyracksTaskContext ctx, int[] fieldSlots, byte[] tupleData,
             int tupleSize) {
         super();
         this.fieldSlots = fieldSlots;
@@ -46,6 +46,7 @@
         appender.reset(writeBuffer, true);
         if (fieldSlots != null && tupleData != null && tupleSize > 0)
             appender.append(fieldSlots, tupleData, 0, tupleSize);
+        writer.open();
         FrameUtils.flushFrame(writeBuffer, writer);
         writer.close();
     }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
index 23c6385..e8a6eae 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
@@ -16,9 +16,9 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -43,14 +43,14 @@
     }
 
     @Override
-    public void contributeTaskGraph(IActivityGraphBuilder builder) {
-        MaterializerActivityNode ma = new MaterializerActivityNode();
-        ReaderActivityNode ra = new ReaderActivityNode();
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        MaterializerActivityNode ma = new MaterializerActivityNode(new ActivityId(odId, 0));
+        ReaderActivityNode ra = new ReaderActivityNode(new ActivityId(odId, 1));
 
-        builder.addTask(ma);
+        builder.addActivity(ma);
         builder.addSourceEdge(0, ma, 0);
 
-        builder.addTask(ra);
+        builder.addActivity(ra);
         builder.addTargetEdge(0, ra, 0);
 
         builder.addBlockingEdge(ma, ra);
@@ -59,16 +59,19 @@
     private final class MaterializerActivityNode extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
+        public MaterializerActivityNode(ActivityId id) {
+            super(id);
+        }
+
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             return new AbstractUnaryInputSinkOperatorNodePushable() {
                 private RunFileWriter out;
 
                 @Override
                 public void open() throws HyracksDataException {
-                    FileReference file = ctx.getJobletContext().createWorkspaceFile(
+                    FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
                             MaterializingOperatorDescriptor.class.getSimpleName());
                     out = new RunFileWriter(file, ctx.getIOManager());
                     out.open();
@@ -90,20 +93,18 @@
                 }
             };
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return MaterializingOperatorDescriptor.this;
-        }
     }
 
     private final class ReaderActivityNode extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
+        public ReaderActivityNode(ActivityId id) {
+            super(id);
+        }
+
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             return new AbstractUnaryOutputSourceOperatorNodePushable() {
                 @Override
                 public void initialize() throws HyracksDataException {
@@ -127,10 +128,5 @@
                 }
             };
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return MaterializingOperatorDescriptor.this;
-        }
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
index 4bc4a7f..5a4ea37 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -33,7 +33,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new AbstractUnaryInputSinkOperatorNodePushable() {
             @Override
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
index e44e8a9..2177409 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.misc;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -58,7 +58,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new DeserializedOperatorNodePushable(ctx, new PrinterOperator(),
                 recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
index f82a7ef..ff39cc8 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
@@ -3,7 +3,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -25,7 +25,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IOperatorEnvironment env,
             final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
             throws HyracksDataException {
         return new AbstractUnaryInputOperatorNodePushable() {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
index 5b1acd5..8a3addb 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
@@ -17,11 +17,10 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -37,18 +36,17 @@
     private class CollectActivity extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
+        public CollectActivity(ActivityId id) {
+            super(id);
+        }
+
         @Override
-        public ActivityNodeId getActivityNodeId() {
+        public ActivityId getActivityId() {
             return id;
         }
 
         @Override
-        public IOperatorDescriptor getOwner() {
-            return SplitVectorOperatorDescriptor.this;
-        }
-
-        @Override
-        public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, final IOperatorEnvironment env,
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, final IOperatorEnvironment env,
                 IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
                 private ArrayList<Object[]> buffer;
@@ -82,13 +80,12 @@
     private class SplitActivity extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return SplitVectorOperatorDescriptor.this;
+        public SplitActivity(ActivityId id) {
+            super(id);
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, final IOperatorEnvironment env,
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, final IOperatorEnvironment env,
                 IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
                 private IOpenableDataWriter<Object[]> writer;
@@ -139,14 +136,14 @@
     }
 
     @Override
-    public void contributeTaskGraph(IActivityGraphBuilder builder) {
-        CollectActivity ca = new CollectActivity();
-        SplitActivity sa = new SplitActivity();
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        CollectActivity ca = new CollectActivity(new ActivityId(odId, 0));
+        SplitActivity sa = new SplitActivity(new ActivityId(odId, 1));
 
-        builder.addTask(ca);
+        builder.addActivity(ca);
         builder.addSourceEdge(0, ca, 0);
 
-        builder.addTask(sa);
+        builder.addActivity(sa);
         builder.addTargetEdge(0, sa, 0);
 
         builder.addBlockingEdge(ca, sa);
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
index fd1d96e..3bde4c4 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -17,10 +17,12 @@
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -28,7 +30,6 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
@@ -64,14 +65,14 @@
     }
 
     @Override
-    public void contributeTaskGraph(IActivityGraphBuilder builder) {
-        SortActivity sa = new SortActivity();
-        MergeActivity ma = new MergeActivity();
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        SortActivity sa = new SortActivity(new ActivityId(odId, 0));
+        MergeActivity ma = new MergeActivity(new ActivityId(odId, 1));
 
-        builder.addTask(sa);
+        builder.addActivity(sa);
         builder.addSourceEdge(0, sa, 0);
 
-        builder.addTask(ma);
+        builder.addActivity(ma);
         builder.addTargetEdge(0, ma, 0);
 
         builder.addBlockingEdge(sa, ma);
@@ -80,15 +81,13 @@
     private class SortActivity extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return ExternalSortOperatorDescriptor.this;
+        public SortActivity(ActivityId id) {
+            super(id);
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             final ExternalSortRunGenerator runGen = new ExternalSortRunGenerator(ctx, sortFields,
                     firstKeyNormalizerFactory, comparatorFactories, recordDescriptors[0], framesLimit);
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
@@ -121,22 +120,24 @@
     private class MergeActivity extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return ExternalSortOperatorDescriptor.this;
+        public MergeActivity(ActivityId id) {
+            super(id);
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
                 @Override
                 public void initialize() throws HyracksDataException {
-                    List<RunFileReader> runs = (List<RunFileReader>) env.get(RUNS);
+                    List<IFrameReader> runs = (List<IFrameReader>) env.get(RUNS);
                     FrameSorter frameSorter = (FrameSorter) env.get(FRAMESORTER);
+                    IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+                    for (int i = 0; i < comparatorFactories.length; ++i) {
+                        comparators[i] = comparatorFactories[i].createBinaryComparator();
+                    }
                     ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, frameSorter, runs, sortFields,
-                            comparatorFactories, recordDescriptors[0], framesLimit, writer);
+                            comparators, recordDescriptors[0], framesLimit, writer);
                     merger.process();
                     env.set(FRAMESORTER, null);
                     env.set(RUNS, null);
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
index 50b968a..eee6f49 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
@@ -18,28 +18,28 @@
 import java.util.LinkedList;
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
 import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
 
 public class ExternalSortRunGenerator implements IFrameWriter {
-    private final IHyracksStageletContext ctx;
+    private final IHyracksTaskContext ctx;
     private final FrameSorter frameSorter;
-    private final List<RunFileReader> runs;
+    private final List<IFrameReader> runs;
     private final int maxSortFrames;
 
-    public ExternalSortRunGenerator(IHyracksStageletContext ctx, int[] sortFields,
+    public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDesc, int framesLimit) {
         this.ctx = ctx;
         frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc);
-        runs = new LinkedList<RunFileReader>();
+        runs = new LinkedList<IFrameReader>();
         maxSortFrames = framesLimit - 1;
     }
 
@@ -70,7 +70,8 @@
 
     private void flushFramesToRun() throws HyracksDataException {
         frameSorter.sortFrames();
-        FileReference file = ctx.getJobletContext().createWorkspaceFile(ExternalSortRunGenerator.class.getSimpleName());
+        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
+                ExternalSortRunGenerator.class.getSimpleName());
         RunFileWriter writer = new RunFileWriter(file, ctx.getIOManager());
         writer.open();
         try {
@@ -90,7 +91,7 @@
         return frameSorter;
     }
 
-    public List<RunFileReader> getRuns() {
+    public List<IFrameReader> getRuns() {
         return runs;
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
index 19f3b7c..8b8429c 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
@@ -16,29 +16,24 @@
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.Comparator;
 import java.util.List;
 
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
 import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
-import edu.uci.ics.hyracks.dataflow.std.util.ReferenceEntry;
-import edu.uci.ics.hyracks.dataflow.std.util.ReferencedPriorityQueue;
 
 public class ExternalSortRunMerger {
-    private final IHyracksStageletContext ctx;
+    private final IHyracksTaskContext ctx;
     private final FrameSorter frameSorter;
-    private final List<RunFileReader> runs;
+    private final List<IFrameReader> runs;
     private final int[] sortFields;
     private final IBinaryComparator[] comparators;
     private final RecordDescriptor recordDesc;
@@ -48,26 +43,21 @@
     private ByteBuffer outFrame;
     private FrameTupleAppender outFrameAppender;
 
-    public ExternalSortRunMerger(IHyracksStageletContext ctx, FrameSorter frameSorter, List<RunFileReader> runs,
-            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDesc,
-            int framesLimit, IFrameWriter writer) {
+    public ExternalSortRunMerger(IHyracksTaskContext ctx, FrameSorter frameSorter, List<IFrameReader> runs,
+            int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDesc, int framesLimit,
+            IFrameWriter writer) {
         this.ctx = ctx;
         this.frameSorter = frameSorter;
         this.runs = runs;
         this.sortFields = sortFields;
-        comparators = new IBinaryComparator[comparatorFactories.length];
-        for (int i = 0; i < comparatorFactories.length; ++i) {
-            comparators[i] = comparatorFactories[i].createBinaryComparator();
-        }
+        this.comparators = comparators;
         this.recordDesc = recordDesc;
         this.framesLimit = framesLimit;
         this.writer = writer;
     }
 
-    public void process(boolean doFinalPass) throws HyracksDataException {
-        if (doFinalPass) {
-            writer.open();
-        }
+    public void process() throws HyracksDataException {
+        writer.open();
         try {
             if (runs.size() <= 0) {
                 if (frameSorter != null && frameSorter.getFrameCount() > 0) {
@@ -81,86 +71,48 @@
                 for (int i = 0; i < framesLimit - 1; ++i) {
                     inFrames.add(ctx.allocateFrame());
                 }
-                int passCount = 0;
                 while (runs.size() > 0) {
-                    passCount++;
                     try {
-                        doPass(runs, passCount, doFinalPass);
+                        doPass(runs);
                     } catch (Exception e) {
                         throw new HyracksDataException(e);
                     }
                 }
             }
         } finally {
-            if (doFinalPass) {
-                writer.close();
-            }
+            writer.close();
         }
     }
 
-    public void process() throws HyracksDataException {
-        process(true);
-    }
-
     // creates a new run from runs that can fit in memory.
-    private void doPass(List<RunFileReader> runs, int passCount, boolean doFinalPass) throws HyracksDataException {
+    private void doPass(List<IFrameReader> runs) throws HyracksDataException {
         FileReference newRun = null;
         IFrameWriter writer = this.writer;
         boolean finalPass = false;
         if (runs.size() + 1 <= framesLimit) { // + 1 outFrame
-            if (!doFinalPass) {
-                return;
-            }
             finalPass = true;
             for (int i = inFrames.size() - 1; i >= runs.size(); i--) {
                 inFrames.remove(i);
             }
         } else {
-            newRun = ctx.createWorkspaceFile(ExternalSortRunMerger.class.getSimpleName());
+            newRun = ctx.createManagedWorkspaceFile(ExternalSortRunMerger.class.getSimpleName());
             writer = new RunFileWriter(newRun, ctx.getIOManager());
             writer.open();
         }
         try {
-            RunFileReader[] runCursors = new RunFileReader[inFrames.size()];
-            FrameTupleAccessor[] tupleAccessors = new FrameTupleAccessor[inFrames.size()];
-            Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
-            ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc,
-                    inFrames.size(), comparator);
-            int[] tupleIndexes = new int[inFrames.size()];
+            IFrameReader[] runCursors = new RunFileReader[inFrames.size()];
             for (int i = 0; i < inFrames.size(); i++) {
-                tupleIndexes[i] = 0;
-                int runIndex = topTuples.peek().getRunid();
-                runCursors[runIndex] = runs.get(runIndex);
-                runCursors[runIndex].open();
-                if (runCursors[runIndex].nextFrame(inFrames.get(runIndex))) {
-                    tupleAccessors[runIndex] = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
-                    tupleAccessors[runIndex].reset(inFrames.get(runIndex));
-                    setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
-                } else {
-                    closeRun(runIndex, runCursors, tupleAccessors);
-                }
+                runCursors[i] = runs.get(i);
             }
-
-            while (!topTuples.areRunsExhausted()) {
-                ReferenceEntry top = topTuples.peek();
-                int runIndex = top.getRunid();
-                FrameTupleAccessor fta = top.getAccessor();
-                int tupleIndex = top.getTupleIndex();
-
-                if (!outFrameAppender.append(fta, tupleIndex)) {
+            RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields,
+                    comparators, recordDesc);
+            merger.open();
+            try {
+                while (merger.nextFrame(outFrame)) {
                     FrameUtils.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) {
-                FrameUtils.flushFrame(outFrame, writer);
-                outFrameAppender.reset(outFrame, true);
+            } finally {
+                merger.close();
             }
             runs.subList(0, inFrames.size()).clear();
             if (!finalPass) {
@@ -172,66 +124,4 @@
             }
         }
     }
-
-    private void setNextTopTuple(int runIndex, int[] tupleIndexes, RunFileReader[] runCursors,
-            FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws HyracksDataException {
-        boolean exists = hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
-        if (exists) {
-            topTuples.popAndReplace(tupleAccessors[runIndex], tupleIndexes[runIndex]);
-        } else {
-            topTuples.pop();
-            closeRun(runIndex, runCursors, tupleAccessors);
-        }
-    }
-
-    private boolean hasNextTuple(int runIndex, int[] tupleIndexes, RunFileReader[] runCursors,
-            FrameTupleAccessor[] tupleAccessors) throws HyracksDataException {
-        if (tupleAccessors[runIndex] == null || runCursors[runIndex] == null) {
-            return false;
-        } else if (tupleIndexes[runIndex] >= tupleAccessors[runIndex].getTupleCount()) {
-            ByteBuffer buf = tupleAccessors[runIndex].getBuffer(); // same-as-inFrames.get(runIndex)
-            if (runCursors[runIndex].nextFrame(buf)) {
-                tupleIndexes[runIndex] = 0;
-                return hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
-            } else {
-                return false;
-            }
-        } else {
-            return true;
-        }
-    }
-
-    private void closeRun(int index, RunFileReader[] runCursors, IFrameTupleAccessor[] tupleAccessor)
-            throws HyracksDataException {
-        runCursors[index].close();
-        runCursors[index] = null;
-        tupleAccessor[index] = null;
-    }
-
-    private Comparator<ReferenceEntry> createEntryComparator(final IBinaryComparator[] comparators) {
-        return new Comparator<ReferenceEntry>() {
-            public int compare(ReferenceEntry tp1, ReferenceEntry tp2) {
-                FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
-                FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
-                int j1 = tp1.getTupleIndex();
-                int j2 = tp2.getTupleIndex();
-                byte[] b1 = fta1.getBuffer().array();
-                byte[] b2 = fta2.getBuffer().array();
-                for (int f = 0; f < sortFields.length; ++f) {
-                    int fIdx = sortFields[f];
-                    int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
-                            + fta1.getFieldStartOffset(j1, fIdx);
-                    int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
-                    int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
-                            + fta2.getFieldStartOffset(j2, fIdx);
-                    int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
-                    int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
-                    if (c != 0) {
-                        return c;
-                    }
-                }
-                return 0;
-            }
-        };
-    }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
index 710ef38..8cf782e 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
@@ -121,7 +121,6 @@
     }
 
     public void flushFrames(IFrameWriter writer) throws HyracksDataException {
-        writer.open();
         appender.reset(outFrame, true);
         for (int ptr = 0; ptr < tupleCount; ++ptr) {
             int i = tPointers[ptr * 4];
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
index 3d41e18..d3015b7 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
@@ -16,9 +16,9 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 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.INormalizedKeyComputerFactory;
@@ -56,14 +56,14 @@
     }
 
     @Override
-    public void contributeTaskGraph(IActivityGraphBuilder builder) {
-        SortActivity sa = new SortActivity();
-        MergeActivity ma = new MergeActivity();
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        SortActivity sa = new SortActivity(new ActivityId(odId, 0));
+        MergeActivity ma = new MergeActivity(new ActivityId(odId, 1));
 
-        builder.addTask(sa);
+        builder.addActivity(sa);
         builder.addSourceEdge(0, sa, 0);
 
-        builder.addTask(ma);
+        builder.addActivity(ma);
         builder.addTargetEdge(0, ma, 0);
 
         builder.addBlockingEdge(sa, ma);
@@ -72,15 +72,13 @@
     private class SortActivity extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return InMemorySortOperatorDescriptor.this;
+        public SortActivity(ActivityId id) {
+            super(id);
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             final FrameSorter frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory,
                     comparatorFactories, recordDescriptors[0]);
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
@@ -111,15 +109,13 @@
     private class MergeActivity extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return InMemorySortOperatorDescriptor.this;
+        public MergeActivity(ActivityId id) {
+            super(id);
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
-                final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
-                int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
             IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
                 @Override
                 public void initialize() throws HyracksDataException {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
new file mode 100644
index 0000000..e791582
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
@@ -0,0 +1,178 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.std.util.ReferenceEntry;
+import edu.uci.ics.hyracks.dataflow.std.util.ReferencedPriorityQueue;
+
+public class RunMergingFrameReader implements IFrameReader {
+    private final IHyracksTaskContext ctx;
+
+    private final IFrameReader[] runCursors;
+
+    private final List<ByteBuffer> inFrames;
+
+    private final int[] sortFields;
+
+    private final IBinaryComparator[] comparators;
+
+    private final RecordDescriptor recordDesc;
+
+    private final FrameTupleAppender outFrameAppender;
+
+    private ReferencedPriorityQueue topTuples;
+
+    private int[] tupleIndexes;
+
+    private FrameTupleAccessor[] tupleAccessors;
+
+    public RunMergingFrameReader(IHyracksTaskContext ctx, IFrameReader[] runCursors, List<ByteBuffer> inFrames,
+            int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDesc) {
+        this.ctx = ctx;
+        this.runCursors = runCursors;
+        this.inFrames = inFrames;
+        this.sortFields = sortFields;
+        this.comparators = comparators;
+        this.recordDesc = recordDesc;
+        outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        tupleAccessors = new FrameTupleAccessor[inFrames.size()];
+        Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
+        topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc, inFrames.size(), comparator);
+        tupleIndexes = new int[inFrames.size()];
+        for (int i = 0; i < inFrames.size(); i++) {
+            tupleIndexes[i] = 0;
+            int runIndex = topTuples.peek().getRunid();
+            runCursors[runIndex].open();
+            if (runCursors[runIndex].nextFrame(inFrames.get(runIndex))) {
+                tupleAccessors[runIndex] = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
+                tupleAccessors[runIndex].reset(inFrames.get(runIndex));
+                setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+            } else {
+                closeRun(runIndex, runCursors, tupleAccessors);
+                topTuples.pop();
+            }
+        }
+    }
+
+    @Override
+    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        outFrameAppender.reset(buffer, true);
+        while (!topTuples.areRunsExhausted()) {
+            ReferenceEntry top = topTuples.peek();
+            int runIndex = top.getRunid();
+            FrameTupleAccessor fta = top.getAccessor();
+            int tupleIndex = top.getTupleIndex();
+
+            if (!outFrameAppender.append(fta, tupleIndex)) {
+                return true;
+            }
+
+            ++tupleIndexes[runIndex];
+            setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+        }
+
+        if (outFrameAppender.getTupleCount() > 0) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        for (int i = 0; i < inFrames.size(); ++i) {
+            closeRun(i, runCursors, tupleAccessors);
+        }
+    }
+
+    private void setNextTopTuple(int runIndex, int[] tupleIndexes, IFrameReader[] runCursors,
+            FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws HyracksDataException {
+        boolean exists = hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+        if (exists) {
+            topTuples.popAndReplace(tupleAccessors[runIndex], tupleIndexes[runIndex]);
+        } else {
+            topTuples.pop();
+            closeRun(runIndex, runCursors, tupleAccessors);
+        }
+    }
+
+    private boolean hasNextTuple(int runIndex, int[] tupleIndexes, IFrameReader[] runCursors,
+            FrameTupleAccessor[] tupleAccessors) throws HyracksDataException {
+        if (tupleAccessors[runIndex] == null || runCursors[runIndex] == null) {
+            return false;
+        } else if (tupleIndexes[runIndex] >= tupleAccessors[runIndex].getTupleCount()) {
+            ByteBuffer buf = tupleAccessors[runIndex].getBuffer(); // same-as-inFrames.get(runIndex)
+            if (runCursors[runIndex].nextFrame(buf)) {
+                tupleIndexes[runIndex] = 0;
+                return hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+            } else {
+                return false;
+            }
+        } else {
+            return true;
+        }
+    }
+
+    private void closeRun(int index, IFrameReader[] runCursors, IFrameTupleAccessor[] tupleAccessors)
+            throws HyracksDataException {
+        if (runCursors[index] != null) {
+            runCursors[index].close();
+            runCursors[index] = null;
+            tupleAccessors[index] = null;
+        }
+    }
+
+    private Comparator<ReferenceEntry> createEntryComparator(final IBinaryComparator[] comparators) {
+        return new Comparator<ReferenceEntry>() {
+            public int compare(ReferenceEntry tp1, ReferenceEntry tp2) {
+                FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
+                FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
+                int j1 = tp1.getTupleIndex();
+                int j2 = tp2.getTupleIndex();
+                byte[] b1 = fta1.getBuffer().array();
+                byte[] b2 = fta2.getBuffer().array();
+                for (int f = 0; f < sortFields.length; ++f) {
+                    int fIdx = sortFields[f];
+                    int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
+                            + fta1.getFieldStartOffset(j1, fIdx);
+                    int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
+                    int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
+                            + fta2.getFieldStartOffset(j2, fIdx);
+                    int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
+                    int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+                    if (c != 0) {
+                        return c;
+                    }
+                }
+                return 0;
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
index 9e8cf00..0acafee 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
@@ -3,7 +3,7 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 /**
  * An entry in the table is: #elements, #no-empty elements; fIndex, tIndex;
@@ -17,14 +17,14 @@
     private IntSerDeBuffer[] headers;
     private List<IntSerDeBuffer> contents = new ArrayList<IntSerDeBuffer>();
     private List<Integer> frameCurrentIndex = new ArrayList<Integer>();
-    private final IHyracksStageletContext ctx;
+    private final IHyracksTaskContext ctx;
     private int frameCapacity = 0;
     private int currentLargestFrameIndex = 0;
     private int tupleCount = 0;
     private int headerFrameCount = 0;
     private TuplePointer tempTuplePointer = new TuplePointer();
 
-    public SerializableHashTable(int tableSize, final IHyracksStageletContext ctx) {
+    public SerializableHashTable(int tableSize, final IHyracksTaskContext ctx) {
         this.ctx = ctx;
         int frameSize = ctx.getFrameSize();
 
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
index eccb945..f5ecde6 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
@@ -17,9 +17,9 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -39,9 +39,9 @@
     private static final long serialVersionUID = 1L;
 
     @Override
-    public void contributeTaskGraph(IActivityGraphBuilder builder) {
-        UnionActivityNode uba = new UnionActivityNode();
-        builder.addTask(uba);
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        UnionActivityNode uba = new UnionActivityNode(new ActivityId(getOperatorId(), 0));
+        builder.addActivity(uba);
         for (int i = 0; i < inputArity; ++i) {
             builder.addSourceEdge(i, uba, i);
         }
@@ -51,22 +51,17 @@
     private class UnionActivityNode extends AbstractActivityNode {
         private static final long serialVersionUID = 1L;
 
-        public UnionActivityNode() {
+        public UnionActivityNode(ActivityId id) {
+            super(id);
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
                 IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
                 throws HyracksDataException {
             RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
             return new UnionOperator(ctx, inRecordDesc);
         }
-
-        @Override
-        public IOperatorDescriptor getOwner() {
-            return UnionAllOperatorDescriptor.this;
-        }
-
     }
 
     private class UnionOperator extends AbstractUnaryOutputOperatorNodePushable {
@@ -74,7 +69,7 @@
 
         private int nClosed;
 
-        public UnionOperator(IHyracksStageletContext ctx, RecordDescriptor inRecordDesc) {
+        public UnionOperator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc) {
             nOpened = 0;
             nClosed = 0;
         }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
index 6660a60..b2c6feb 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
@@ -17,7 +17,7 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameDeserializer;
@@ -26,13 +26,13 @@
 import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
 
 public final class DeserializedOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
-    private final IHyracksStageletContext ctx;
+    private final IHyracksTaskContext ctx;
 
     private final IOpenableDataWriterOperator delegate;
 
     private final FrameDeserializer deserializer;
 
-    public DeserializedOperatorNodePushable(IHyracksStageletContext ctx, IOpenableDataWriterOperator delegate,
+    public DeserializedOperatorNodePushable(IHyracksTaskContext ctx, IOpenableDataWriterOperator delegate,
             RecordDescriptor inRecordDesc) {
         this.ctx = ctx;
         this.delegate = delegate;
diff --git a/hyracks-documentation/.classpath b/hyracks-documentation/.classpath
index d0bec0f..3f62785 100644
--- a/hyracks-documentation/.classpath
+++ b/hyracks-documentation/.classpath
@@ -1,6 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <classpath>
-	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.5"/>
+	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.4"/>
 	<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
 	<classpathentry kind="output" path="target/classes"/>
 </classpath>
diff --git a/hyracks-documentation/.settings/org.eclipse.jdt.core.prefs b/hyracks-documentation/.settings/org.eclipse.jdt.core.prefs
index 75a3d16..f362c73 100644
--- a/hyracks-documentation/.settings/org.eclipse.jdt.core.prefs
+++ b/hyracks-documentation/.settings/org.eclipse.jdt.core.prefs
@@ -1,6 +1,6 @@
-#Thu Aug 04 11:50:30 PDT 2011
+#Sun Aug 14 10:18:14 PDT 2011
 eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.5
-org.eclipse.jdt.core.compiler.compliance=1.5
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.4
+org.eclipse.jdt.core.compiler.compliance=1.4
 org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
-org.eclipse.jdt.core.compiler.source=1.5
+org.eclipse.jdt.core.compiler.source=1.4
diff --git a/hyracks-examples/btree-example/btreeapp/.classpath b/hyracks-examples/btree-example/btreeapp/.classpath
index d0bec0f..3f62785 100644
--- a/hyracks-examples/btree-example/btreeapp/.classpath
+++ b/hyracks-examples/btree-example/btreeapp/.classpath
@@ -1,6 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <classpath>
-	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.5"/>
+	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.4"/>
 	<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
 	<classpathentry kind="output" path="target/classes"/>
 </classpath>
diff --git a/hyracks-examples/btree-example/btreeapp/.settings/org.eclipse.jdt.core.prefs b/hyracks-examples/btree-example/btreeapp/.settings/org.eclipse.jdt.core.prefs
index 75a3d16..f362c73 100644
--- a/hyracks-examples/btree-example/btreeapp/.settings/org.eclipse.jdt.core.prefs
+++ b/hyracks-examples/btree-example/btreeapp/.settings/org.eclipse.jdt.core.prefs
@@ -1,6 +1,6 @@
-#Thu Aug 04 11:50:30 PDT 2011
+#Sun Aug 14 10:18:14 PDT 2011
 eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.5
-org.eclipse.jdt.core.compiler.compliance=1.5
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.4
+org.eclipse.jdt.core.compiler.compliance=1.4
 org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
-org.eclipse.jdt.core.compiler.source=1.5
+org.eclipse.jdt.core.compiler.source=1.4
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
index 7aeaf77..6f371c0 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
@@ -15,8 +15,6 @@
 
 package edu.uci.ics.hyracks.examples.btree.client;
 
-import java.util.UUID;
-
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
@@ -30,6 +28,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
@@ -37,7 +36,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
@@ -92,7 +91,7 @@
         JobSpecification job = createJob(options);
 
         long start = System.currentTimeMillis();
-        UUID jobId = hcc.createJob(options.app, job);
+        JobId jobId = hcc.createJob(options.app, job);
         hcc.start(jobId);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
@@ -202,7 +201,7 @@
         // ops
         IBinaryHashFunctionFactory[] hashFactories = new IBinaryHashFunctionFactory[1];
         hashFactories[0] = UTF8StringBinaryHashFunctionFactory.INSTANCE;
-        IConnectorDescriptor hashConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 }, hashFactories));
 
         // connect the ops
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index 1533714..f7942eb 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -14,8 +14,6 @@
  */
 package edu.uci.ics.hyracks.examples.btree.client;
 
-import java.util.UUID;
-
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
@@ -29,13 +27,14 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
@@ -89,7 +88,7 @@
         JobSpecification job = createJob(options);
 
         long start = System.currentTimeMillis();
-        UUID jobId = hcc.createJob(options.app, job);
+        JobId jobId = hcc.createJob(options.app, job);
         hcc.start(jobId);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
@@ -167,7 +166,7 @@
         // ops
         IBinaryHashFunctionFactory[] hashFactories = new IBinaryHashFunctionFactory[1];
         hashFactories[0] = UTF8StringBinaryHashFunctionFactory.INSTANCE;
-        IConnectorDescriptor hashConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 }, hashFactories));
 
         spec.connect(hashConn, dataGen, 0, sorter, 0);
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
index c991a89..cae9660 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
@@ -14,8 +14,6 @@
  */
 package edu.uci.ics.hyracks.examples.btree.client;
 
-import java.util.UUID;
-
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
@@ -26,6 +24,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
@@ -74,7 +73,7 @@
         JobSpecification job = createJob(options);
 
         long start = System.currentTimeMillis();
-        UUID jobId = hcc.createJob(options.app, job);
+        JobId jobId = hcc.createJob(options.app, job);
         hcc.start(jobId);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index 7e4ae9d..fbf7f9d 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -15,7 +15,6 @@
 package edu.uci.ics.hyracks.examples.btree.client;
 
 import java.io.DataOutput;
-import java.util.UUID;
 
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
@@ -28,6 +27,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
@@ -81,7 +81,7 @@
         JobSpecification job = createJob(options);
 
         long start = System.currentTimeMillis();
-        UUID jobId = hcc.createJob(options.app, job);
+        JobId jobId = hcc.createJob(options.app, job);
         hcc.start(jobId);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index b061811..47fecad 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -14,8 +14,6 @@
  */
 package edu.uci.ics.hyracks.examples.btree.client;
 
-import java.util.UUID;
-
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
 
@@ -26,6 +24,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
@@ -85,7 +84,7 @@
         JobSpecification job = createJob(options);
 
         long start = System.currentTimeMillis();
-        UUID jobId = hcc.createJob(options.app, job);
+        JobId jobId = hcc.createJob(options.app, job);
         hcc.start(jobId);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 5e5eaeb..5eec60c 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -15,7 +15,6 @@
 package edu.uci.ics.hyracks.examples.btree.client;
 
 import java.io.DataOutput;
-import java.util.UUID;
 
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
@@ -28,6 +27,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
@@ -84,7 +84,7 @@
         JobSpecification job = createJob(options);
 
         long start = System.currentTimeMillis();
-        UUID jobId = hcc.createJob(options.app, job);
+        JobId jobId = hcc.createJob(options.app, job);
         hcc.start(jobId);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
index 0c72503..357760a 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
@@ -20,7 +20,7 @@
 import java.util.HashSet;
 import java.util.Random;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -59,7 +59,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
 
         final ByteBuffer outputFrame = ctx.allocateFrame();
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
index 9fd7df4..4a452a8 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
@@ -16,7 +16,7 @@
 package edu.uci.ics.hyracks.examples.btree.helper;
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
 import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
@@ -58,7 +58,7 @@
         return treeIndexRegistry;
     }
     
-    public static RuntimeContext get(IHyracksStageletContext ctx) {
+    public static RuntimeContext get(IHyracksTaskContext ctx) {
         return (RuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
     }
 }
\ No newline at end of file
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java
index e1e47af..4c6363b 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.examples.btree.helper;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -29,12 +29,12 @@
     }
 
     @Override
-    public IBufferCache getBufferCache(IHyracksStageletContext ctx) {
+    public IBufferCache getBufferCache(IHyracksTaskContext ctx) {
         return RuntimeContext.get(ctx).getBufferCache();
     }
 
     @Override
-    public IFileMapProvider getFileMapProvider(IHyracksStageletContext ctx) {
+    public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
         return RuntimeContext.get(ctx).getFileMapManager();
     }
 }
\ No newline at end of file
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java
index 6a6ddf8..f8a7bf1 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.examples.btree.helper;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
@@ -29,7 +29,7 @@
     }
 
     @Override
-    public IndexRegistry<ITreeIndex> getRegistry(IHyracksStageletContext ctx) {
+    public IndexRegistry<ITreeIndex> getRegistry(IHyracksTaskContext ctx) {
         return RuntimeContext.get(ctx).getTreeIndexRegistry();
     }
 }
\ No newline at end of file
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
index f13a701..99204d8 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
@@ -19,16 +19,12 @@
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.UUID;
 
 import org.junit.Test;
 
-import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
-import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
@@ -40,25 +36,19 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.SerializingDataWriter;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.test.support.TestJobletContext;
-import edu.uci.ics.hyracks.test.support.TestNCApplicationContext;
-import edu.uci.ics.hyracks.test.support.TestRootContext;
-import edu.uci.ics.hyracks.test.support.TestStageletContext;
+import edu.uci.ics.hyracks.test.support.TestUtils;
 
 public class SerializationDeserializationTest {
     private static final String DBLP_FILE = "data/dblp.txt";
 
     private static class SerDeserRunner {
-        private final IHyracksStageletContext ctx;
+        private final IHyracksTaskContext ctx;
         private static final int FRAME_SIZE = 32768;
         private RecordDescriptor rDes;
         private List<ByteBuffer> buffers;
 
         public SerDeserRunner(RecordDescriptor rDes) throws HyracksException {
-            IHyracksRootContext rootCtx = new TestRootContext(FRAME_SIZE);
-            INCApplicationContext appCtx = new TestNCApplicationContext(rootCtx, null);
-            IHyracksJobletContext jobletCtx = new TestJobletContext(appCtx, UUID.randomUUID(), 0);
-            ctx = new TestStageletContext(jobletCtx, UUID.randomUUID());
+            ctx = TestUtils.create(FRAME_SIZE);
             this.rDes = rDes;
             buffers = new ArrayList<ByteBuffer>();
         }
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index 2bb1d80..237ab52 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -15,18 +15,18 @@
 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.client.HyracksLocalConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.control.CCConfig;
-import edu.uci.ics.hyracks.api.control.NCConfig;
 import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
 
 public abstract class AbstractIntegrationTest {
@@ -42,7 +42,7 @@
     public static void init() throws Exception {
         CCConfig ccConfig = new CCConfig();
         ccConfig.port = 39001;
-        ccConfig.profileDumpPeriod = 1000;
+        ccConfig.profileDumpPeriod = 10000;
         cc = new ClusterControllerService(ccConfig);
         cc.start();
 
@@ -74,8 +74,8 @@
     }
 
     protected void runTest(JobSpecification spec) throws Exception {
-        UUID jobId = hcc.createJob("test", spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
-        System.err.println(spec.toJSON());
+        JobId jobId = hcc.createJob("test", spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+        System.err.println(spec.toJSON().toString(2));
         hcc.start(jobId);
         System.err.print(jobId);
         cc.waitForCompletion(jobId);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
index 8468b54..0a0df60 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
@@ -37,7 +37,7 @@
 import edu.uci.ics.hyracks.dataflow.std.aggregators.CountAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.aggregators.IFieldValueResultingAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
@@ -95,7 +95,7 @@
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(conn1, csvScanner, 0, sorter, 0);
@@ -103,7 +103,7 @@
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, sorter, 0, group, 0);
 
-        IConnectorDescriptor conn3 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn3 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(conn3, group, 0, sorter2, 0);
@@ -162,7 +162,7 @@
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(conn1, csvScanner, 0, sorter, 0);
@@ -170,7 +170,7 @@
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, sorter, 0, group, 0);
 
-        IConnectorDescriptor conn3 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn3 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(conn3, group, 0, sorter2, 0);
@@ -229,7 +229,7 @@
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(conn1, csvScanner, 0, sorter, 0);
@@ -237,7 +237,7 @@
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, sorter, 0, group, 0);
 
-        IConnectorDescriptor conn3 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn3 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(conn3, group, 0, sorter2, 0);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
index f0974d9..d343232 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
@@ -33,7 +33,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -96,7 +96,7 @@
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(conn1, ordScanner, 0, printer, 0);
@@ -130,7 +130,7 @@
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
                         new IBinaryHashFunctionFactory[] { IntegerBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(conn1, ordScanner, 0, printer, 0);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
index b0c33ae..1b6ff14 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -31,7 +31,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningMergingConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -75,7 +75,7 @@
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
 
-        spec.connect(new MToNHashPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
+        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
                 new int[] { 1 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
                 new int[] { 1 }, new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }),
                 sorter, 0, printer, 0);
@@ -116,7 +116,7 @@
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
 
-        spec.connect(new MToNHashPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
+        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
                 new int[] { 1, 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE,
                         UTF8StringBinaryHashFunctionFactory.INSTANCE }), new int[] { 1, 0 },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index a8db325..d417fda 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -38,7 +38,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
@@ -621,12 +621,12 @@
                 : new NullSinkOperatorDescriptor(spec);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
-        IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
 
-        IConnectorDescriptor custJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(custJoinConn, custScanner, 0, join, 1);
@@ -700,12 +700,12 @@
                 : new NullSinkOperatorDescriptor(spec);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
-        IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
 
-        IConnectorDescriptor custJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(custJoinConn, custScanner, 0, join, 1);
@@ -779,12 +779,12 @@
                 : new NullSinkOperatorDescriptor(spec);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
-        IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
 
-        IConnectorDescriptor custJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(custJoinConn, custScanner, 0, join, 1);
@@ -857,12 +857,12 @@
                 : new NullSinkOperatorDescriptor(spec);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
-        IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
 
-        IConnectorDescriptor custJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(custJoinConn, custScanner, 0, join, 1);
@@ -941,12 +941,12 @@
                 : new NullSinkOperatorDescriptor(spec);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
-        IConnectorDescriptor ordPartConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor ordPartConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(ordPartConn, ordScanner, 0, ordMat, 0);
 
-        IConnectorDescriptor custPartConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor custPartConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(custPartConn, custScanner, 0, custMat, 0);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java
index b107282..2f66bfd 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java
@@ -44,7 +44,7 @@
 import edu.uci.ics.hyracks.dataflow.std.aggregators.IntSumAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -144,7 +144,7 @@
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
@@ -188,7 +188,7 @@
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
                         UTF8StringBinaryHashFunctionFactory.INSTANCE, UTF8StringBinaryHashFunctionFactory.INSTANCE, }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
@@ -234,7 +234,7 @@
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
                         UTF8StringBinaryHashFunctionFactory.INSTANCE, UTF8StringBinaryHashFunctionFactory.INSTANCE, }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
@@ -276,7 +276,7 @@
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
@@ -325,7 +325,7 @@
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
                         UTF8StringBinaryHashFunctionFactory.INSTANCE, UTF8StringBinaryHashFunctionFactory.INSTANCE, }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
@@ -367,7 +367,7 @@
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
diff --git a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
index b02b0ee..68e33cb 100644
--- a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
+++ b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
@@ -15,7 +15,6 @@
 package edu.uci.ics.hyracks.examples.text.client;
 
 import java.io.File;
-import java.util.UUID;
 
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
@@ -30,6 +29,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.hash.IntegerBinaryHashFunctionFactory;
@@ -51,7 +51,7 @@
 import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -122,7 +122,7 @@
 
             System.out.print(i + "\t" + (System.currentTimeMillis() - start));
             start = System.currentTimeMillis();
-            UUID jobId = hcc.createJob(options.app, job);
+            JobId jobId = hcc.createJob(options.app, job);
             hcc.start(jobId);
             hcc.waitForCompletion(jobId);
             System.out.println("\t" + (System.currentTimeMillis() - start));
@@ -221,7 +221,7 @@
                 createPartitionConstraint(spec, grouper, outSplits);
 
                 // Connect scanner with the grouper
-                IConnectorDescriptor scanGroupConn = new MToNHashPartitioningConnectorDescriptor(spec,
+                IConnectorDescriptor scanGroupConn = new MToNPartitioningConnectorDescriptor(spec,
                         new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
                         // IntegerBinaryHashFunctionFactory.INSTANCE,
                                 IntegerBinaryHashFunctionFactory.INSTANCE }));
@@ -235,7 +235,7 @@
                 createPartitionConstraint(spec, sorter, inSplits);
 
                 // Connect scan operator with the sorter
-                IConnectorDescriptor scanSortConn = new MToNHashPartitioningConnectorDescriptor(spec,
+                IConnectorDescriptor scanSortConn = new MToNPartitioningConnectorDescriptor(spec,
                         new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
                         // IntegerBinaryHashFunctionFactory.INSTANCE,
                                 IntegerBinaryHashFunctionFactory.INSTANCE }));
@@ -265,7 +265,7 @@
                 createPartitionConstraint(spec, grouper, outSplits);
 
                 // Connect scanner with the grouper
-                IConnectorDescriptor scanConn = new MToNHashPartitioningConnectorDescriptor(spec,
+                IConnectorDescriptor scanConn = new MToNPartitioningConnectorDescriptor(spec,
                         new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
                         // IntegerBinaryHashFunctionFactory.INSTANCE,
                                 IntegerBinaryHashFunctionFactory.INSTANCE }));
@@ -292,11 +292,11 @@
                 createPartitionConstraint(spec, grouper, outSplits);
 
                 // Connect scanner with the grouper
-                IConnectorDescriptor defaultGroupConn = new MToNHashPartitioningConnectorDescriptor(spec,
+                IConnectorDescriptor scanGroupConnDef = new MToNPartitioningConnectorDescriptor(spec,
                         new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
                         // IntegerBinaryHashFunctionFactory.INSTANCE,
                                 IntegerBinaryHashFunctionFactory.INSTANCE }));
-                spec.connect(defaultGroupConn, fileScanner, 0, grouper, 0);
+                spec.connect(scanGroupConnDef, fileScanner, 0, grouper, 0);
         }
 
         IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(outSplits);
diff --git a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
index a0f24c4..51d3661 100644
--- a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
+++ b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
@@ -15,7 +15,6 @@
 package edu.uci.ics.hyracks.examples.text.client;
 
 import java.io.File;
-import java.util.UUID;
 
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
@@ -30,6 +29,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
@@ -40,7 +40,7 @@
 import edu.uci.ics.hyracks.dataflow.std.aggregators.CountAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.aggregators.IFieldValueResultingAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
@@ -95,7 +95,7 @@
                 options.algo, options.htSize, options.sbSize, options.format);
 
         long start = System.currentTimeMillis();
-        UUID jobId = hcc.createJob(options.app, job);
+        JobId jobId = hcc.createJob(options.app, job);
         hcc.start(jobId);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
@@ -141,7 +141,7 @@
                             new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                     groupResultDesc, htSize);
             createPartitionConstraint(spec, gBy, outSplits);
-            IConnectorDescriptor scanGroupConn = new MToNHashPartitioningConnectorDescriptor(spec,
+            IConnectorDescriptor scanGroupConn = new MToNPartitioningConnectorDescriptor(spec,
                     new FieldHashPartitionComputerFactory(keys,
                             new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
             spec.connect(scanGroupConn, wordScanner, 0, gBy, 0);
@@ -153,7 +153,7 @@
                             new UTF8StringNormalizedKeyComputerFactory(), cfs, wordDesc);
             createPartitionConstraint(spec, sorter, outSplits);
 
-            IConnectorDescriptor scanSortConn = new MToNHashPartitioningConnectorDescriptor(spec,
+            IConnectorDescriptor scanSortConn = new MToNPartitioningConnectorDescriptor(spec,
                     new FieldHashPartitionComputerFactory(keys,
                             new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
             spec.connect(scanSortConn, wordScanner, 0, sorter, 0);
diff --git a/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java b/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
index 4d7cc1f..249e3bb 100644
--- a/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
+++ b/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
@@ -9,7 +9,7 @@
 import java.util.Arrays;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -23,7 +23,7 @@
     private static final long serialVersionUID = 1L;
 
     @Override
-    public ITupleParser createTupleParser(final IHyracksStageletContext ctx) {
+    public ITupleParser createTupleParser(final IHyracksTaskContext ctx) {
         return new ITupleParser() {
             @Override
             public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
diff --git a/hyracks-examples/tpch-example/tpchapp/.classpath b/hyracks-examples/tpch-example/tpchapp/.classpath
index d0bec0f..3f62785 100644
--- a/hyracks-examples/tpch-example/tpchapp/.classpath
+++ b/hyracks-examples/tpch-example/tpchapp/.classpath
@@ -1,6 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <classpath>
-	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.5"/>
+	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.4"/>
 	<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
 	<classpathentry kind="output" path="target/classes"/>
 </classpath>
diff --git a/hyracks-examples/tpch-example/tpchapp/.settings/org.eclipse.jdt.core.prefs b/hyracks-examples/tpch-example/tpchapp/.settings/org.eclipse.jdt.core.prefs
index b61b886..f362c73 100644
--- a/hyracks-examples/tpch-example/tpchapp/.settings/org.eclipse.jdt.core.prefs
+++ b/hyracks-examples/tpch-example/tpchapp/.settings/org.eclipse.jdt.core.prefs
@@ -1,6 +1,6 @@
-#Thu Aug 04 11:50:35 PDT 2011
+#Sun Aug 14 10:18:14 PDT 2011
 eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.5
-org.eclipse.jdt.core.compiler.compliance=1.5
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.4
+org.eclipse.jdt.core.compiler.compliance=1.4
 org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
-org.eclipse.jdt.core.compiler.source=1.5
+org.eclipse.jdt.core.compiler.source=1.4
diff --git a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index b1a1224..09d0e5a 100644
--- a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -16,7 +16,6 @@
 
 import java.io.File;
 import java.util.EnumSet;
-import java.util.UUID;
 
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
@@ -32,6 +31,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
@@ -43,7 +43,7 @@
 import edu.uci.ics.hyracks.dataflow.std.aggregators.CountAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.aggregators.IFieldValueResultingAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -93,7 +93,7 @@
                 options.numJoinPartitions);
 
         long start = System.currentTimeMillis();
-        UUID jobId = hcc.createJob(options.app, job,
+        JobId jobId = hcc.createJob(options.app, job,
                 options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
         hcc.start(jobId);
         hcc.waitForCompletion(jobId);
@@ -184,17 +184,17 @@
         FrameFileWriterOperatorDescriptor writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
         createPartitionConstraint(spec, writer, resultSplits);
 
-        IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(ordJoinConn, ordScanner, 0, join, 1);
 
-        IConnectorDescriptor custJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(custJoinConn, custScanner, 0, join, 0);
 
-        IConnectorDescriptor joinGroupConn = new MToNHashPartitioningConnectorDescriptor(spec,
+        IConnectorDescriptor joinGroupConn = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 6 },
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
         spec.connect(joinGroupConn, join, 0, gby, 0);
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
index a363221..9e76a3e 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
@@ -4,10 +4,10 @@
 import java.util.HashSet;
 import java.util.Properties;
 import java.util.Set;
-import java.util.UUID;
 
 import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
 import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.hadoop.compat.util.ConfigurationConstants;
@@ -15,77 +15,73 @@
 
 public class HyracksClient {
 
-	private static HyracksRMIConnection connection;
-	private static final String jobProfilingKey = "jobProfilingKey";
-	Set<String> systemLibs;
+    private static HyracksRMIConnection connection;
+    private static final String jobProfilingKey = "jobProfilingKey";
+    Set<String> systemLibs;
 
-	public HyracksClient(Properties clusterProperties) throws Exception {
-		initialize(clusterProperties);
-	}
+    public HyracksClient(Properties clusterProperties) throws Exception {
+        initialize(clusterProperties);
+    }
 
-	private void initialize(Properties properties) throws Exception {
-		String clusterController = (String) properties
-				.get(ConfigurationConstants.clusterControllerHost);
-		connection = new HyracksRMIConnection(clusterController, 1099);
-		systemLibs = new HashSet<String>();
-		for (String systemLib : ConfigurationConstants.systemLibs) {
-			String systemLibPath = properties.getProperty(systemLib);
-			if (systemLibPath != null) {
-				systemLibs.add(systemLibPath);
-			}
-		}
-	}
+    private void initialize(Properties properties) throws Exception {
+        String clusterController = (String) properties.get(ConfigurationConstants.clusterControllerHost);
+        connection = new HyracksRMIConnection(clusterController, 1099);
+        systemLibs = new HashSet<String>();
+        for (String systemLib : ConfigurationConstants.systemLibs) {
+            String systemLibPath = properties.getProperty(systemLib);
+            if (systemLibPath != null) {
+                systemLibs.add(systemLibPath);
+            }
+        }
+    }
 
-	public HyracksClient(String clusterConf, char delimiter) throws Exception {
-		Properties properties = Utilities.getProperties(clusterConf, delimiter);
-		initialize(properties);
-	}
+    public HyracksClient(String clusterConf, char delimiter) throws Exception {
+        Properties properties = Utilities.getProperties(clusterConf, delimiter);
+        initialize(properties);
+    }
 
-	private Set<String> getRequiredLibs(Set<String> userLibs) {
-		Set<String> requiredLibs = new HashSet<String>();
-		for (String systemLib : systemLibs) {
-			requiredLibs.add(systemLib);
-		}
-		for (String userLib : userLibs) {
-			requiredLibs.add(userLib);
-		}
-		return requiredLibs;
-	}
+    private Set<String> getRequiredLibs(Set<String> userLibs) {
+        Set<String> requiredLibs = new HashSet<String>();
+        for (String systemLib : systemLibs) {
+            requiredLibs.add(systemLib);
+        }
+        for (String userLib : userLibs) {
+            requiredLibs.add(userLib);
+        }
+        return requiredLibs;
+    }
 
-	public JobStatus getJobStatus(UUID jobId) throws Exception {
-		return connection.getJobStatus(jobId);
-	}
+    public JobStatus getJobStatus(JobId jobId) throws Exception {
+        return connection.getJobStatus(jobId);
+    }
 
-	private void createApplication(String applicationName, Set<String> userLibs)
-			throws Exception {
-		connection.createApplication(applicationName, Utilities
-				.getHyracksArchive(applicationName, getRequiredLibs(userLibs)));
-	}
+    private void createApplication(String applicationName, Set<String> userLibs) throws Exception {
+        connection.createApplication(applicationName,
+                Utilities.getHyracksArchive(applicationName, getRequiredLibs(userLibs)));
+    }
 
-	public HyracksRunningJob submitJob(String applicationName,
-			JobSpecification spec) throws Exception {
-		String jobProfilingVal = System.getenv(jobProfilingKey);
-		boolean doProfiling = ("true".equalsIgnoreCase(jobProfilingVal));
-		UUID jobId;
-		if (doProfiling) {
-			System.out.println("PROFILING");
-			jobId = connection.createJob(applicationName, spec, EnumSet
-					.of(JobFlag.PROFILE_RUNTIME));
-		} else {
-			jobId = connection.createJob(applicationName, spec);
-		}
-		connection.start(jobId);
-		HyracksRunningJob runningJob = new HyracksRunningJob(jobId, spec, this);
-		return runningJob;
-	}
+    public HyracksRunningJob submitJob(String applicationName, JobSpecification spec) throws Exception {
+        String jobProfilingVal = System.getenv(jobProfilingKey);
+        boolean doProfiling = ("true".equalsIgnoreCase(jobProfilingVal));
+        JobId jobId;
+        if (doProfiling) {
+            System.out.println("PROFILING");
+            jobId = connection.createJob(applicationName, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+        } else {
+            jobId = connection.createJob(applicationName, spec);
+        }
+        connection.start(jobId);
+        HyracksRunningJob runningJob = new HyracksRunningJob(jobId, spec, this);
+        return runningJob;
+    }
 
-	public HyracksRunningJob submitJob(String applicationName,
-			JobSpecification spec, Set<String> userLibs) throws Exception {
-		createApplication(applicationName, userLibs);
-		return submitJob(applicationName, spec);
-	}
+    public HyracksRunningJob submitJob(String applicationName, JobSpecification spec, Set<String> userLibs)
+            throws Exception {
+        createApplication(applicationName, userLibs);
+        return submitJob(applicationName, spec);
+    }
 
-	public void waitForCompleton(UUID jobId) throws Exception {
-		connection.waitForCompletion(jobId);
-	}
+    public void waitForCompleton(JobId jobId) throws Exception {
+        connection.waitForCompletion(jobId);
+    }
 }
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksRunningJob.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksRunningJob.java
index 8470e12..7c650ec 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksRunningJob.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksRunningJob.java
@@ -1,7 +1,6 @@
 package edu.uci.ics.hyracks.hadoop.compat.client;
 
 import java.io.IOException;
-import java.util.UUID;
 
 import org.apache.hadoop.mapred.Counters;
 import org.apache.hadoop.mapred.JobID;
@@ -9,19 +8,20 @@
 import org.apache.hadoop.mapred.TaskAttemptID;
 import org.apache.hadoop.mapred.TaskCompletionEvent;
 
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 public class HyracksRunningJob implements RunningJob {
 
-    UUID jobId;
+    JobId jobId;
     JobSpecification spec;
     HyracksClient hyracksClient;
 
-    public UUID getJobId() {
+    public JobId getJobId() {
         return jobId;
     }
 
-    public void setJobId(UUID jobId) {
+    public void setJobId(JobId jobId) {
         this.jobId = jobId;
     }
 
@@ -33,7 +33,7 @@
         this.spec = spec;
     }
 
-    public HyracksRunningJob(UUID jobId, JobSpecification jobSpec, HyracksClient hyracksClient) {
+    public HyracksRunningJob(JobId jobId, JobSpecification jobSpec, HyracksClient hyracksClient) {
         this.spec = jobSpec;
         this.jobId = jobId;
         this.hyracksClient = hyracksClient;
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/driver/CompatibilityLayer.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/driver/CompatibilityLayer.java
index 37f4d34..bf09bb0 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/driver/CompatibilityLayer.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/driver/CompatibilityLayer.java
@@ -6,15 +6,16 @@
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
-import java.util.UUID;
-import java.util.Map.Entry;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobConf;
 import org.kohsuke.args4j.CmdLineParser;
 
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.hadoop.compat.client.HyracksClient;
 import edu.uci.ics.hyracks.hadoop.compat.client.HyracksRunningJob;
 import edu.uci.ics.hyracks.hadoop.compat.util.CompatibilityConfig;
@@ -22,187 +23,168 @@
 import edu.uci.ics.hyracks.hadoop.compat.util.DCacheHandler;
 import edu.uci.ics.hyracks.hadoop.compat.util.HadoopAdapter;
 import edu.uci.ics.hyracks.hadoop.compat.util.Utilities;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 public class CompatibilityLayer {
 
-	HyracksClient hyracksClient;
-	DCacheHandler dCacheHander = null;
-	Properties clusterConf;
-	HadoopAdapter hadoopAdapter;
+    HyracksClient hyracksClient;
+    DCacheHandler dCacheHander = null;
+    Properties clusterConf;
+    HadoopAdapter hadoopAdapter;
 
-	private static char configurationFileDelimiter = '=';
-	private static final String dacheKeyPrefix = "dcache.key";
+    private static char configurationFileDelimiter = '=';
+    private static final String dacheKeyPrefix = "dcache.key";
 
-	public CompatibilityLayer(CompatibilityConfig clConfig) throws Exception {
-		initialize(clConfig);
-	}
+    public CompatibilityLayer(CompatibilityConfig clConfig) throws Exception {
+        initialize(clConfig);
+    }
 
-	private void initialize(CompatibilityConfig clConfig) throws Exception {
-		clusterConf = Utilities.getProperties(clConfig.clusterConf,
-				configurationFileDelimiter);
-		hadoopAdapter = new HadoopAdapter(clusterConf
-				.getProperty(ConfigurationConstants.namenodeURL));
-		hyracksClient = new HyracksClient(clusterConf);
-		dCacheHander = new DCacheHandler(clusterConf
-				.getProperty(ConfigurationConstants.dcacheServerConfiguration));
-	}
+    private void initialize(CompatibilityConfig clConfig) throws Exception {
+        clusterConf = Utilities.getProperties(clConfig.clusterConf, configurationFileDelimiter);
+        hadoopAdapter = new HadoopAdapter(clusterConf.getProperty(ConfigurationConstants.namenodeURL));
+        hyracksClient = new HyracksClient(clusterConf);
+        dCacheHander = new DCacheHandler(clusterConf.getProperty(ConfigurationConstants.dcacheServerConfiguration));
+    }
 
-	public HyracksRunningJob submitJob(JobConf conf,Set<String> userLibs) throws Exception {
-		List<JobConf> jobConfs = new ArrayList<JobConf>();
-		jobConfs.add(conf);
-		String applicationName = conf.getJobName() + System.currentTimeMillis();
-		JobSpecification spec = hadoopAdapter.getJobSpecification(jobConfs);
-		HyracksRunningJob hyracksRunningJob = hyracksClient.submitJob(
-				applicationName, spec, userLibs);
-		return hyracksRunningJob; 
-	}
-	
-	public HyracksRunningJob submitJobs(String applicationName,
-			String[] jobFiles, Set<String> userLibs) throws Exception {
-		List<JobConf> jobConfs = constructHadoopJobConfs(jobFiles);
-		populateDCache(jobFiles[0]);
-		JobSpecification spec = hadoopAdapter.getJobSpecification(jobConfs);
-		HyracksRunningJob hyracksRunningJob = hyracksClient.submitJob(
-				applicationName, spec, userLibs);
-		return hyracksRunningJob;
-	}
+    public HyracksRunningJob submitJob(JobConf conf, Set<String> userLibs) throws Exception {
+        List<JobConf> jobConfs = new ArrayList<JobConf>();
+        jobConfs.add(conf);
+        String applicationName = conf.getJobName() + System.currentTimeMillis();
+        JobSpecification spec = hadoopAdapter.getJobSpecification(jobConfs);
+        HyracksRunningJob hyracksRunningJob = hyracksClient.submitJob(applicationName, spec, userLibs);
+        return hyracksRunningJob;
+    }
 
-	public HyracksRunningJob submitJobs(String applicationName,
-			String[] jobFiles) throws Exception {
-		List<JobConf> jobConfs = constructHadoopJobConfs(jobFiles);
-		populateDCache(jobFiles[0]);
-		JobSpecification spec = hadoopAdapter.getJobSpecification(jobConfs);
-		HyracksRunningJob hyracksRunningJob = hyracksClient.submitJob(
-				applicationName, spec);
-		return hyracksRunningJob;
-	}
+    public HyracksRunningJob submitJobs(String applicationName, String[] jobFiles, Set<String> userLibs)
+            throws Exception {
+        List<JobConf> jobConfs = constructHadoopJobConfs(jobFiles);
+        populateDCache(jobFiles[0]);
+        JobSpecification spec = hadoopAdapter.getJobSpecification(jobConfs);
+        HyracksRunningJob hyracksRunningJob = hyracksClient.submitJob(applicationName, spec, userLibs);
+        return hyracksRunningJob;
+    }
 
-	private void populateDCache(String jobFile) throws IOException {
-		Map<String, String> dcacheTasks = preparePreLaunchDCacheTasks(jobFile);
-		String tempDir = "/tmp";
-		if (dcacheTasks.size() > 0) {
-			for (String key : dcacheTasks.keySet()) {
-				String destPath = tempDir + "/" + key
-						+ System.currentTimeMillis();
-				hadoopAdapter.getHDFSClient().copyToLocalFile(
-						new Path(dcacheTasks.get(key)), new Path(destPath));
-				System.out.println(" source :" + dcacheTasks.get(key));
-				System.out.println(" dest :" + destPath);
-				System.out.println(" key :" + key);
-				System.out.println(" value :" + destPath);
-				dCacheHander.put(key, destPath);
-			}
-		}
-	}
+    public HyracksRunningJob submitJobs(String applicationName, String[] jobFiles) throws Exception {
+        List<JobConf> jobConfs = constructHadoopJobConfs(jobFiles);
+        populateDCache(jobFiles[0]);
+        JobSpecification spec = hadoopAdapter.getJobSpecification(jobConfs);
+        HyracksRunningJob hyracksRunningJob = hyracksClient.submitJob(applicationName, spec);
+        return hyracksRunningJob;
+    }
 
-	private String getApplicationNameForHadoopJob(JobConf jobConf) {
-		String jar = jobConf.getJar();
-		if (jar != null) {
-			return jar.substring(jar.lastIndexOf("/") >= 0 ? jar
-					.lastIndexOf("/") + 1 : 0);
-		} else {
-			return "" + System.currentTimeMillis();
-		}
-	}
+    private void populateDCache(String jobFile) throws IOException {
+        Map<String, String> dcacheTasks = preparePreLaunchDCacheTasks(jobFile);
+        String tempDir = "/tmp";
+        if (dcacheTasks.size() > 0) {
+            for (String key : dcacheTasks.keySet()) {
+                String destPath = tempDir + "/" + key + System.currentTimeMillis();
+                hadoopAdapter.getHDFSClient().copyToLocalFile(new Path(dcacheTasks.get(key)), new Path(destPath));
+                System.out.println(" source :" + dcacheTasks.get(key));
+                System.out.println(" dest :" + destPath);
+                System.out.println(" key :" + key);
+                System.out.println(" value :" + destPath);
+                dCacheHander.put(key, destPath);
+            }
+        }
+    }
 
-	private Map<String, String> initializeCustomProperties(
-			Properties properties, String prefix) {
-		Map<String, String> foundProperties = new HashMap<String, String>();
-		Set<Entry<Object, Object>> entrySet = properties.entrySet();
-		for (Entry entry : entrySet) {
-			String key = (String) entry.getKey();
-			String value = (String) entry.getValue();
-			if ((key.startsWith(prefix))) {
-				String actualKey = key.substring(prefix.length() + 1); // "cut off '<prefix>.' from the beginning"
-				foundProperties.put(actualKey, value);
-			}
-		}
-		return foundProperties;
-	}
+    private String getApplicationNameForHadoopJob(JobConf jobConf) {
+        String jar = jobConf.getJar();
+        if (jar != null) {
+            return jar.substring(jar.lastIndexOf("/") >= 0 ? jar.lastIndexOf("/") + 1 : 0);
+        } else {
+            return "" + System.currentTimeMillis();
+        }
+    }
 
-	public Map<String, String> preparePreLaunchDCacheTasks(String jobFile) {
-		Properties jobProperties = Utilities.getProperties(jobFile, ',');
-		Map<String, String> dcacheTasks = new HashMap<String, String>();
-		Map<String, String> dcacheKeys = initializeCustomProperties(
-				jobProperties, dacheKeyPrefix);
-		for (String key : dcacheKeys.keySet()) {
-			String sourcePath = dcacheKeys.get(key);
-			if (sourcePath != null) {
-				dcacheTasks.put(key, sourcePath);
-			}
-		}
-		return dcacheTasks;
-	}
+    private Map<String, String> initializeCustomProperties(Properties properties, String prefix) {
+        Map<String, String> foundProperties = new HashMap<String, String>();
+        Set<Entry<Object, Object>> entrySet = properties.entrySet();
+        for (Entry entry : entrySet) {
+            String key = (String) entry.getKey();
+            String value = (String) entry.getValue();
+            if ((key.startsWith(prefix))) {
+                String actualKey = key.substring(prefix.length() + 1); // "cut off '<prefix>.' from the beginning"
+                foundProperties.put(actualKey, value);
+            }
+        }
+        return foundProperties;
+    }
 
-	public void waitForCompletion(UUID jobId) throws Exception {
-		hyracksClient.waitForCompleton(jobId);
-	}
+    public Map<String, String> preparePreLaunchDCacheTasks(String jobFile) {
+        Properties jobProperties = Utilities.getProperties(jobFile, ',');
+        Map<String, String> dcacheTasks = new HashMap<String, String>();
+        Map<String, String> dcacheKeys = initializeCustomProperties(jobProperties, dacheKeyPrefix);
+        for (String key : dcacheKeys.keySet()) {
+            String sourcePath = dcacheKeys.get(key);
+            if (sourcePath != null) {
+                dcacheTasks.put(key, sourcePath);
+            }
+        }
+        return dcacheTasks;
+    }
 
-	private List<JobConf> constructHadoopJobConfs(String[] jobFiles)
-			throws Exception {
-		List<JobConf> jobConfs = new ArrayList<JobConf>();
-		for (String jobFile : jobFiles) {
-			jobConfs.add(constructHadoopJobConf(jobFile));
-		}
-		return jobConfs;
-	}
+    public void waitForCompletion(JobId jobId) throws Exception {
+        hyracksClient.waitForCompleton(jobId);
+    }
 
-	private JobConf constructHadoopJobConf(String jobFile) {
-		Properties jobProperties = Utilities.getProperties(jobFile, '=');
-		JobConf conf = new JobConf(hadoopAdapter.getConf());
-		for (Entry entry : jobProperties.entrySet()) {
-			conf.set((String) entry.getKey(), (String) entry.getValue());
-			System.out.println((String) entry.getKey() + " : "
-					+ (String) entry.getValue());
-		}
-		return conf;
-	}
+    private List<JobConf> constructHadoopJobConfs(String[] jobFiles) throws Exception {
+        List<JobConf> jobConfs = new ArrayList<JobConf>();
+        for (String jobFile : jobFiles) {
+            jobConfs.add(constructHadoopJobConf(jobFile));
+        }
+        return jobConfs;
+    }
 
-	private String[] getJobs(CompatibilityConfig clConfig) {
-		return clConfig.jobFiles == null ? new String[0] : clConfig.jobFiles
-				.split(",");
-	}
+    private JobConf constructHadoopJobConf(String jobFile) {
+        Properties jobProperties = Utilities.getProperties(jobFile, '=');
+        JobConf conf = new JobConf(hadoopAdapter.getConf());
+        for (Entry entry : jobProperties.entrySet()) {
+            conf.set((String) entry.getKey(), (String) entry.getValue());
+            System.out.println((String) entry.getKey() + " : " + (String) entry.getValue());
+        }
+        return conf;
+    }
 
-	public static void main(String args[]) throws Exception {
-		long startTime = System.nanoTime();
-		CompatibilityConfig clConfig = new CompatibilityConfig();
-		CmdLineParser cp = new CmdLineParser(clConfig);
-		try {
-			cp.parseArgument(args);
-		} catch (Exception e) {
-			System.err.println(e.getMessage());
-			cp.printUsage(System.err);
-			return;
-		}
-		CompatibilityLayer compatLayer = new CompatibilityLayer(clConfig);
-		String applicationName = clConfig.applicationName;
-		String[] jobFiles = compatLayer.getJobs(clConfig);
-		String[] userLibraries = null;
-		if (clConfig.userLibs != null) {
-			userLibraries = clConfig.userLibs.split(",");
-		}
-		try {
-			HyracksRunningJob hyraxRunningJob = null;
-			if (userLibraries != null) {
-				Set<String> userLibs = new HashSet<String>();
-				for (String userLib : userLibraries) {
-					userLibs.add(userLib);
-				}
-				hyraxRunningJob = compatLayer.submitJobs(applicationName,
-						jobFiles, userLibs);
-			} else {
-				hyraxRunningJob = compatLayer.submitJobs(applicationName,
-						jobFiles);
-			}
-			compatLayer.waitForCompletion(hyraxRunningJob.getJobId());
-			long end_time = System.nanoTime();
-			System.out.println("TOTAL TIME (from Launch to Completion):"
-					+ ((end_time - startTime) / (float) 1000000000.0)
-					+ " seconds.");
-		} catch (Exception e) {
-			e.printStackTrace();
-			throw e;
-		}
-	}
+    private String[] getJobs(CompatibilityConfig clConfig) {
+        return clConfig.jobFiles == null ? new String[0] : clConfig.jobFiles.split(",");
+    }
+
+    public static void main(String args[]) throws Exception {
+        long startTime = System.nanoTime();
+        CompatibilityConfig clConfig = new CompatibilityConfig();
+        CmdLineParser cp = new CmdLineParser(clConfig);
+        try {
+            cp.parseArgument(args);
+        } catch (Exception e) {
+            System.err.println(e.getMessage());
+            cp.printUsage(System.err);
+            return;
+        }
+        CompatibilityLayer compatLayer = new CompatibilityLayer(clConfig);
+        String applicationName = clConfig.applicationName;
+        String[] jobFiles = compatLayer.getJobs(clConfig);
+        String[] userLibraries = null;
+        if (clConfig.userLibs != null) {
+            userLibraries = clConfig.userLibs.split(",");
+        }
+        try {
+            HyracksRunningJob hyraxRunningJob = null;
+            if (userLibraries != null) {
+                Set<String> userLibs = new HashSet<String>();
+                for (String userLib : userLibraries) {
+                    userLibs.add(userLib);
+                }
+                hyraxRunningJob = compatLayer.submitJobs(applicationName, jobFiles, userLibs);
+            } else {
+                hyraxRunningJob = compatLayer.submitJobs(applicationName, jobFiles);
+            }
+            compatLayer.waitForCompletion(hyraxRunningJob.getJobId());
+            long end_time = System.nanoTime();
+            System.out.println("TOTAL TIME (from Launch to Completion):"
+                    + ((end_time - startTime) / (float) 1000000000.0) + " seconds.");
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
 }
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/util/HadoopAdapter.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/util/HadoopAdapter.java
index f2f7d03..28833d7 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/util/HadoopAdapter.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/util/HadoopAdapter.java
@@ -40,7 +40,7 @@
 import edu.uci.ics.hyracks.dataflow.hadoop.util.ClasspathBasedHadoopClassFactory;
 import edu.uci.ics.hyracks.dataflow.hadoop.util.DatatypeHelper;
 import edu.uci.ics.hyracks.dataflow.hadoop.util.IHadoopClassFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
@@ -370,13 +370,13 @@
 		return externalSortOp;
 	}
 
-	public static MToNHashPartitioningConnectorDescriptor getMtoNHashPartitioningConnector(
+	public static MToNPartitioningConnectorDescriptor getMtoNHashPartitioningConnector(
 			JobConf conf, JobSpecification spec) {
 
 		Class mapOutputKeyClass = conf.getMapOutputKeyClass();
 		Class mapOutputValueClass = conf.getMapOutputValueClass();
 
-		MToNHashPartitioningConnectorDescriptor connectorDescriptor = null;
+		MToNPartitioningConnectorDescriptor connectorDescriptor = null;
 		ITuplePartitionComputerFactory factory = null;
 		conf.getMapOutputKeyClass();
 		if (conf.getPartitionerClass() != null
@@ -398,7 +398,7 @@
 			factory = new HadoopHashTuplePartitionComputerFactory(
 					mapOutputKeySerializerDerserializer);
 		}
-		connectorDescriptor = new MToNHashPartitioningConnectorDescriptor(spec,
+		connectorDescriptor = new MToNPartitioningConnectorDescriptor(spec,
 				factory);
 		return connectorDescriptor;
 	}
diff --git a/hyracks-server/.classpath b/hyracks-server/.classpath
index d0bec0f..3f62785 100644
--- a/hyracks-server/.classpath
+++ b/hyracks-server/.classpath
@@ -1,6 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <classpath>
-	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.5"/>
+	<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.4"/>
 	<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
 	<classpathentry kind="output" path="target/classes"/>
 </classpath>
diff --git a/hyracks-server/.settings/org.eclipse.jdt.core.prefs b/hyracks-server/.settings/org.eclipse.jdt.core.prefs
index b61b886..f362c73 100644
--- a/hyracks-server/.settings/org.eclipse.jdt.core.prefs
+++ b/hyracks-server/.settings/org.eclipse.jdt.core.prefs
@@ -1,6 +1,6 @@
-#Thu Aug 04 11:50:35 PDT 2011
+#Sun Aug 14 10:18:14 PDT 2011
 eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.5
-org.eclipse.jdt.core.compiler.compliance=1.5
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.4
+org.eclipse.jdt.core.compiler.compliance=1.4
 org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
-org.eclipse.jdt.core.compiler.source=1.5
+org.eclipse.jdt.core.compiler.source=1.4
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
index c85c6a0..9b8d4a2 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
@@ -1,6 +1,6 @@
 package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
@@ -15,18 +15,18 @@
 
 public class BTreeOpHelper extends TreeIndexOpHelper {
 
-    public BTreeOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, IHyracksStageletContext ctx, int partition,
+    public BTreeOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, IHyracksTaskContext ctx, int partition,
             IndexHelperOpenMode mode) {
         super(opDesc, ctx, partition, mode);
     }
-    
+
     public ITreeIndex createTreeIndex() throws HyracksDataException {
         IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
         ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
-        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, indexFileId, 0, metaDataFrameFactory);
+        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, indexFileId, 0,
+                metaDataFrameFactory);
         return new BTree(bufferCache, freePageManager, opDesc.getTreeIndexInteriorFactory(),
                 opDesc.getTreeIndexLeafFactory(), cmp);
     }
-    
-    
-}
+
+}
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java
index 7802b81..6cc79ab 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java
@@ -1,15 +1,10 @@
 package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
-import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
 
 public class BTreeOpHelperFactory implements ITreeIndexOpHelperFactory {
 
@@ -17,7 +12,7 @@
 
     @Override
     public TreeIndexOpHelper createTreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc,
-            IHyracksStageletContext ctx, int partition, IndexHelperOpenMode mode) {
+            IHyracksTaskContext ctx, int partition, IndexHelperOpenMode mode) {
         return new BTreeOpHelper(opDesc, ctx, partition, mode);
     }
 
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 254935b..717c99e 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -58,7 +58,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, final IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new BTreeSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, isForward, lowKeyFields,
                 highKeyFields, lowKeyInclusive, highKeyInclusive);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index 9d33bb2..bfb8ed6 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -17,7 +17,7 @@
 import java.io.DataOutput;
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -31,8 +31,8 @@
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
-import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
@@ -66,10 +66,11 @@
 
     private RecordDescriptor recDesc;
 
-    public BTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+    public BTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, int[] lowKeyFields,
             int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive) {
-        treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.OPEN);
+        treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
+                IndexHelperOpenMode.OPEN);
         this.isForward = isForward;
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
@@ -88,15 +89,15 @@
     public void open() throws HyracksDataException {
         AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
                 .getOperatorDescriptor();
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
 
         cursorFrame = opDesc.getTreeIndexLeafFactory().createFrame();
-        cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame)cursorFrame);
+        cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame);
+        writer.open();
 
         try {
-
             treeIndexOpHelper.init();
-            btree = (BTree)treeIndexOpHelper.getTreeIndex();
+            btree = (BTree) treeIndexOpHelper.getTreeIndex();
 
             // construct range predicate
 
@@ -128,19 +129,20 @@
             rangePred = new RangePredicate(isForward, null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
                     highKeySearchCmp);
 
-            accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+            accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
 
-            writeBuffer = treeIndexOpHelper.getHyracksStageletContext().allocateFrame();
+            writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
             tb = new ArrayTupleBuilder(btree.getMultiComparator().getFieldCount());
             dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksStageletContext().getFrameSize());
+            appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
-            opCtx = btree.createOpContext(IndexOp.SEARCH, treeIndexOpHelper.getLeafFrame(), treeIndexOpHelper
-                    .getInteriorFrame(), null);
+            opCtx = btree.createOpContext(IndexOp.SEARCH, treeIndexOpHelper.getLeafFrame(),
+                    treeIndexOpHelper.getInteriorFrame(), null);
 
         } catch (Exception e) {
             treeIndexOpHelper.deinit();
+            throw new HyracksDataException(e);
         }
     }
 
diff --git a/hyracks-storage-am-common/.settings/org.maven.ide.eclipse.prefs b/hyracks-storage-am-common/.settings/org.maven.ide.eclipse.prefs
index 7b5e618..34ccda1 100644
--- a/hyracks-storage-am-common/.settings/org.maven.ide.eclipse.prefs
+++ b/hyracks-storage-am-common/.settings/org.maven.ide.eclipse.prefs
@@ -1,7 +1,8 @@
-#Thu Jul 07 12:23:53 PDT 2011
+#Mon Aug 15 10:50:16 PDT 2011
 activeProfiles=
 eclipse.preferences.version=1
 fullBuildGoals=process-test-resources
+includeModules=false
 resolveWorkspaceProjects=true
 resourceFilterGoals=process-resources resources\:testResources
 skipCompilerPlugin=true
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexRegistryProvider.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexRegistryProvider.java
index aadcaf9..7971a62 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexRegistryProvider.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexRegistryProvider.java
@@ -17,8 +17,8 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 public interface IIndexRegistryProvider<IndexType> extends Serializable {
-    public IndexRegistry<IndexType> getRegistry(IHyracksStageletContext ctx);
+    public IndexRegistry<IndexType> getRegistry(IHyracksTaskContext ctx);
 }
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java
index fa37fab..cb4d92d 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java
@@ -2,9 +2,9 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
 public interface ITreeIndexOpHelperFactory extends Serializable {
     public TreeIndexOpHelper createTreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc,
-            final IHyracksStageletContext ctx, int partition, IndexHelperOpenMode mode);
+            final IHyracksTaskContext ctx, int partition, IndexHelperOpenMode mode);
 }
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java
index 6ca4529..9098a78 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java
@@ -1,6 +1,6 @@
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -9,7 +9,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
-public interface ITreeIndexOperatorDescriptorHelper extends IActivityNode {
+public interface ITreeIndexOperatorDescriptorHelper extends IActivity {
     public IFileSplitProvider getTreeIndexFileSplitProvider();
 
     public IBinaryComparatorFactory[] getTreeIndexComparatorFactories();
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
index 7d585b3..fc12f59 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -46,7 +46,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new TreeIndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, fillFactor,
                 recordDescProvider);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
index 2001039..817aeb4 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -36,9 +36,8 @@
 
     private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
 
-    public TreeIndexBulkLoadOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
-            IHyracksStageletContext ctx, int partition, int[] fieldPermutation, float fillFactor,
-            IRecordDescriptorProvider recordDescProvider) {
+    public TreeIndexBulkLoadOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition, int[] fieldPermutation, float fillFactor, IRecordDescriptorProvider recordDescProvider) {
         treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
                 IndexHelperOpenMode.CREATE);
         this.fillFactor = fillFactor;
@@ -51,7 +50,7 @@
         AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
                 .getOperatorDescriptor();
         RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
         ITreeIndexMetaDataFrame metaFrame = new LIFOMetaDataFrame();
         try {
             treeIndexOpHelper.init();
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
index 73b5323..3a6eb62 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -40,7 +40,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new TreeIndexDiskOrderScanOperatorNodePushable(this, ctx, partition);
     }
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index 415b9e3..5b5c6e8 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -17,7 +17,7 @@
 import java.io.DataOutput;
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -35,7 +35,7 @@
     private final TreeIndexOpHelper treeIndexOpHelper;
 
     public TreeIndexDiskOrderScanOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
-            IHyracksStageletContext ctx, int partition) {
+            IHyracksTaskContext ctx, int partition) {
         treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
                 IndexHelperOpenMode.OPEN);
     }
@@ -57,8 +57,8 @@
                 treeIndexOpHelper.getTreeIndex().diskOrderScan(cursor, cursorFrame, metaFrame, diskOrderScanOpCtx);
 
                 int fieldCount = treeIndexOpHelper.getTreeIndex().getFieldCount();
-                ByteBuffer frame = treeIndexOpHelper.getHyracksStageletContext().allocateFrame();
-                FrameTupleAppender appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksStageletContext()
+                ByteBuffer frame = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
+                FrameTupleAppender appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext()
                         .getFrameSize());
                 appender.reset(frame, true);
                 ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java
index 2cfa905..2b1de29 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
@@ -42,7 +42,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new TreeIndexDropOperatorNodePushable(ctx, storageManager, treeIndexRegistryProvider, fileSplitProvider,
                 partition);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
index 71346f7..9b4ecb0 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
@@ -19,7 +19,7 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -33,13 +33,13 @@
 public class TreeIndexDropOperatorNodePushable extends AbstractOperatorNodePushable {
     private static final Logger LOGGER = Logger.getLogger(TreeIndexDropOperatorNodePushable.class.getName());
 
-    private final IHyracksStageletContext ctx;
+    private final IHyracksTaskContext ctx;
     private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
     private IStorageManagerInterface storageManager;
     private IFileSplitProvider fileSplitProvider;
     private int partition;
 
-    public TreeIndexDropOperatorNodePushable(IHyracksStageletContext ctx, IStorageManagerInterface storageManager,
+    public TreeIndexDropOperatorNodePushable(IHyracksTaskContext ctx, IStorageManagerInterface storageManager,
             IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, IFileSplitProvider fileSplitProvider,
             int partition) {
         this.ctx = ctx;
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java
index 0961a4f..7812198 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -48,7 +48,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int partitions) throws HyracksDataException {
         return new TreeIndexFileEnlistmentOperatorNodePushable(this, ctx, partition);
     }
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java
index 8ff6586..9cbde53 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java
@@ -16,7 +16,7 @@
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
@@ -26,7 +26,7 @@
     private final TreeIndexOpHelper treeIndexOpHelper;
 
     public TreeIndexFileEnlistmentOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
-            IHyracksStageletContext ctx, int partition) {
+            IHyracksTaskContext ctx, int partition) {
         treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
                 IndexHelperOpenMode.ENLIST);
     }
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
index c8141b9..0457768 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -50,7 +50,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new TreeIndexInsertUpdateDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
                 recordDescProvider, op);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
index 74a9efc..496678b 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -38,7 +38,7 @@
     private IndexOpContext opCtx;
 
     public TreeIndexInsertUpdateDeleteOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
-            IHyracksStageletContext ctx, int partition, int[] fieldPermutation,
+            IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
             IRecordDescriptorProvider recordDescProvider, IndexOp op) {
         treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
                 IndexHelperOpenMode.OPEN);
@@ -52,8 +52,9 @@
         AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
                 .getOperatorDescriptor();
         RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), inputRecDesc);
-        writeBuffer = treeIndexOpHelper.getHyracksStageletContext().allocateFrame();
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
+        writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
+        writer.open();
         try {
             treeIndexOpHelper.init();
             treeIndexOpHelper.getTreeIndex().open(treeIndexOpHelper.getIndexFileId());
@@ -96,6 +97,7 @@
 
             } catch (Exception e) {
                 e.printStackTrace();
+                throw new HyracksDataException(e);
             }
         }
 
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java
index fca6d8d..af98bb3 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -39,12 +39,12 @@
     protected int partition;
 
     protected ITreeIndexOperatorDescriptorHelper opDesc;
-    protected IHyracksStageletContext ctx;
+    protected IHyracksTaskContext ctx;
 
     protected IndexHelperOpenMode mode;
 
-    public TreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, final IHyracksStageletContext ctx,
-            int partition, IndexHelperOpenMode mode) {
+    public TreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, final IHyracksTaskContext ctx, int partition,
+            IndexHelperOpenMode mode) {
         this.opDesc = opDesc;
         this.ctx = ctx;
         this.mode = mode;
@@ -154,7 +154,7 @@
         return treeIndex;
     }
 
-    public IHyracksStageletContext getHyracksStageletContext() {
+    public IHyracksTaskContext getHyracksTaskContext() {
         return ctx;
     }
 
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
index 574e727..d221e25 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
@@ -1,6 +1,6 @@
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -26,7 +26,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new TreeIndexStatsOperatorNodePushable(this, ctx, partition);
     }
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
index f47855f..8c825c2 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
@@ -15,7 +15,7 @@
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
@@ -26,10 +26,10 @@
 
 public class TreeIndexStatsOperatorNodePushable extends AbstractOperatorNodePushable {
     private final TreeIndexOpHelper treeIndexOpHelper;
-    private final IHyracksStageletContext ctx;
+    private final IHyracksTaskContext ctx;
     private TreeIndexStatsGatherer statsGatherer;
 
-    public TreeIndexStatsOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+    public TreeIndexStatsOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition) {
         treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
                 IndexHelperOpenMode.CREATE);
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
index 83246d6..b5b1393 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -47,7 +47,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         return new BinaryTokenizerOperatorNodePushable(ctx, recordDescProvider.getInputRecordDescriptor(odId, 0),
                 recordDescriptors[0], tokenizerFactory.createTokenizer(), tokenFields, projFields);
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index 0647f45..1a0091a 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -19,7 +19,7 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -32,7 +32,7 @@
 
 public class BinaryTokenizerOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
 
-    private final IHyracksStageletContext ctx;
+    private final IHyracksTaskContext ctx;
     private final IBinaryTokenizer tokenizer;
     private final int[] tokenFields;
     private final int[] projFields;
@@ -45,7 +45,7 @@
     private FrameTupleAppender appender;
     private ByteBuffer writeBuffer;
 
-    public BinaryTokenizerOperatorNodePushable(IHyracksStageletContext ctx, RecordDescriptor inputRecDesc,
+    public BinaryTokenizerOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc,
             RecordDescriptor outputRecDesc, IBinaryTokenizer tokenizer, int[] tokenFields, int[] projFields) {
         this.ctx = ctx;
         this.tokenizer = tokenizer;
@@ -63,6 +63,7 @@
         builderDos = builder.getDataOutput();
         appender = new FrameTupleAppender(ctx.getFrameSize());
         appender.reset(writeBuffer, true);
+        writer.open();
     }
 
     @Override
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
index d003580..1ab72fe 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -57,7 +57,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new InvertedIndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, btreeFillFactor,
                 invListBuilder, recordDescProvider);
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
index 4969124..d2bd395 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -36,7 +36,7 @@
     protected final IInvertedListBuilder invListBuilder;
     private InvertedIndex.BulkLoadContext bulkLoadCtx;
 
-    private final IHyracksStageletContext ctx;
+    private final IHyracksTaskContext ctx;
 
     private FrameTupleAccessor accessor;
     private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
@@ -44,7 +44,7 @@
     private IRecordDescriptorProvider recordDescProvider;
 
     public InvertedIndexBulkLoadOperatorNodePushable(AbstractInvertedIndexOperatorDescriptor opDesc,
-            IHyracksStageletContext ctx, int partition, int[] fieldPermutation, float btreeFillFactor,
+            IHyracksTaskContext ctx, int partition, int[] fieldPermutation, float btreeFillFactor,
             IInvertedListBuilder invListBuilder, IRecordDescriptorProvider recordDescProvider) {
         treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
                 IndexHelperOpenMode.CREATE);
@@ -61,7 +61,7 @@
         AbstractInvertedIndexOperatorDescriptor opDesc = (AbstractInvertedIndexOperatorDescriptor) treeIndexOpHelper
                 .getOperatorDescriptor();
         RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
 
         // btree
         try {
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java
index c16cfcd..1eb6757 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -37,11 +37,11 @@
     private int partition;
 
     private IInvertedIndexOperatorDescriptorHelper opDesc;
-    private IHyracksStageletContext ctx;
+    private IHyracksTaskContext ctx;
 
     private IndexHelperOpenMode mode;
 
-    public InvertedIndexOpHelper(IInvertedIndexOperatorDescriptorHelper opDesc, final IHyracksStageletContext ctx,
+    public InvertedIndexOpHelper(IInvertedIndexOperatorDescriptorHelper opDesc, final IHyracksTaskContext ctx,
             int partition, IndexHelperOpenMode mode) {
         this.opDesc = opDesc;
         this.ctx = ctx;
@@ -144,10 +144,6 @@
         return invIndex;
     }
 
-    public IHyracksStageletContext getHyracksStageletContext() {
-        return ctx;
-    }
-
     public ITreeIndexOperatorDescriptorHelper getOperatorDescriptor() {
         return opDesc;
     }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
index d1fba3b..7505ae0 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
@@ -22,7 +22,7 @@
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -52,7 +52,7 @@
 
 public class TOccurrenceSearcher implements IInvertedIndexSearcher {
 
-    protected final IHyracksStageletContext ctx;
+    protected final IHyracksTaskContext ctx;
     protected final FixedSizeFrameTupleAppender resultFrameTupleApp;
     protected final FixedSizeFrameTupleAccessor resultFrameTupleAcc;
     protected final FixedSizeTupleReference resultTuple;
@@ -87,7 +87,7 @@
     protected List<IInvertedListCursor> invListCursorCache = new ArrayList<IInvertedListCursor>(cursorCacheSize);
     protected List<IInvertedListCursor> invListCursors = new ArrayList<IInvertedListCursor>(cursorCacheSize);
 
-    public TOccurrenceSearcher(IHyracksStageletContext ctx, InvertedIndex invIndex, IBinaryTokenizer queryTokenizer) {
+    public TOccurrenceSearcher(IHyracksTaskContext ctx, InvertedIndex invIndex, IBinaryTokenizer queryTokenizer) {
         this.ctx = ctx;
         this.invIndex = invIndex;
         this.queryTokenizer = queryTokenizer;
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java
index 30d67f0..0f5439b 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java
@@ -19,7 +19,7 @@
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
@@ -27,7 +27,7 @@
 
 public class TOccurrenceSearcherSuffixProbeOnly extends TOccurrenceSearcher {
 
-    public TOccurrenceSearcherSuffixProbeOnly(IHyracksStageletContext ctx, InvertedIndex invIndex,
+    public TOccurrenceSearcherSuffixProbeOnly(IHyracksTaskContext ctx, InvertedIndex invIndex,
             IBinaryTokenizer queryTokenizer) {
         super(ctx, invIndex, queryTokenizer);
     }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java
index f8bc1ab..b997dc9 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java
@@ -19,7 +19,7 @@
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
@@ -28,7 +28,7 @@
 
 public class TOccurrenceSearcherSuffixScanOnly extends TOccurrenceSearcher {
 
-    public TOccurrenceSearcherSuffixScanOnly(IHyracksStageletContext ctx, InvertedIndex invIndex,
+    public TOccurrenceSearcherSuffixScanOnly(IHyracksTaskContext ctx, InvertedIndex invIndex,
             IBinaryTokenizer queryTokenizer) {
         super(ctx, invIndex, queryTokenizer);
     }
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelper.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelper.java
index 5b8931d..2fcb065 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelper.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelper.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.rtree.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
@@ -33,7 +33,7 @@
 
     protected MultiComparator interiorCmp;
 
-    public RTreeOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, IHyracksStageletContext ctx, int partition,
+    public RTreeOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, IHyracksTaskContext ctx, int partition,
             IndexHelperOpenMode mode) {
         super(opDesc, ctx, partition, mode);
     }
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelperFactory.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelperFactory.java
index b26f21a..75668e1 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelperFactory.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelperFactory.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.rtree.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
@@ -27,7 +27,7 @@
 
     @Override
     public TreeIndexOpHelper createTreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc,
-            IHyracksStageletContext ctx, int partition, IndexHelperOpenMode mode) {
+            IHyracksTaskContext ctx, int partition, IndexHelperOpenMode mode) {
         return new RTreeOpHelper(opDesc, ctx, partition, mode);
     }
 }
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 8aab763..3dc0963 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -15,7 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.rtree.dataflow;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -48,7 +48,7 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, final IOperatorEnvironment env,
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new RTreeSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, keyFields);
     }
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 6a13d9f..86c5591 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -18,7 +18,7 @@
 import java.io.DataOutput;
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -64,7 +64,7 @@
 
     private RecordDescriptor recDesc;
 
-    public RTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+    public RTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition, IRecordDescriptorProvider recordDescProvider, int[] keyFields) {
         treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
                 IndexHelperOpenMode.OPEN);
@@ -80,12 +80,14 @@
     public void open() throws HyracksDataException {
         AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
                 .getOperatorDescriptor();
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
 
         interiorFrame = opDesc.getTreeIndexInteriorFactory().createFrame();
         leafFrame = opDesc.getTreeIndexLeafFactory().createFrame();
         cursor = new RTreeSearchCursor((IRTreeInteriorFrame) interiorFrame, (IRTreeLeafFrame) leafFrame);
 
+        writer.open();
+
         try {
 
             treeIndexOpHelper.init();
@@ -100,12 +102,12 @@
             cmp = new MultiComparator(rtree.getCmp().getTypeTraits(), keySearchComparators);
 
             searchPred = new SearchPredicate(searchKey, cmp);
-            accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+            accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
 
-            writeBuffer = treeIndexOpHelper.getHyracksStageletContext().allocateFrame();
+            writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
             tb = new ArrayTupleBuilder(rtree.getCmp().getFieldCount());
             dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksStageletContext().getFrameSize());
+            appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
             opCtx = rtree.createOpContext(IndexOp.SEARCH, treeIndexOpHelper.getLeafFrame(),
@@ -113,6 +115,7 @@
 
         } catch (Exception e) {
             treeIndexOpHelper.deinit();
+            throw new HyracksDataException(e);
         }
     }
 
diff --git a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java
index f64af2e..562305e 100644
--- a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java
+++ b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java
@@ -16,12 +16,12 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public interface IStorageManagerInterface extends Serializable {
-    public IBufferCache getBufferCache(IHyracksStageletContext ctx);
+    public IBufferCache getBufferCache(IHyracksTaskContext ctx);
 
-    public IFileMapProvider getFileMapProvider(IHyracksStageletContext ctx);
+    public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx);
 }
\ No newline at end of file
diff --git a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
index f9ac4d5..d7558ba 100644
--- a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
@@ -321,6 +321,39 @@
         }
     }
 
+    private String dumpState() {
+        StringBuilder buffer = new StringBuilder();
+        buffer.append("Buffer cache state\n");
+        buffer.append("Page Size: ").append(pageSize).append('\n');
+        buffer.append("Number of physical pages: ").append(numPages).append('\n');
+        buffer.append("Hash table size: ").append(pageMap.length).append('\n');
+        buffer.append("Page Map:\n");
+        int nCachedPages = 0;
+        for (int i = 0; i < pageMap.length; ++i) {
+            CacheBucket cb = pageMap[i];
+            cb.bucketLock.lock();
+            try {
+                CachedPage cp = cb.cachedPage;
+                if (cp != null) {
+                    buffer.append("   ").append(i).append('\n');
+                    while (cp != null) {
+                        buffer.append("      ").append(cp.cpid).append(" -> [")
+                                .append(BufferedFileHandle.getFileId(cp.dpid)).append(':')
+                                .append(BufferedFileHandle.getPageId(cp.dpid)).append(", ").append(cp.pinCount.get())
+                                .append(", ").append(cp.valid ? "valid" : "invalid").append(", ")
+                                .append(cp.dirty.get() ? "dirty" : "clean").append("]\n");
+                        cp = cp.next;
+                        ++nCachedPages;
+                    }
+                }
+            } finally {
+                cb.bucketLock.unlock();
+            }
+        }
+        buffer.append("Number of cached pages: ").append(nCachedPages).append('\n');
+        return buffer.toString();
+    }
+
     private void read(CachedPage cPage) throws HyracksDataException {
         BufferedFileHandle fInfo = getFileInfo(cPage);
         cPage.buffer.clear();
@@ -645,6 +678,7 @@
     public void closeFile(int fileId) throws HyracksDataException {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Closing file: " + fileId + " in cache: " + this);
+            LOGGER.info(dumpState());
         }
         synchronized (fileInfoMap) {
             BufferedFileHandle fInfo = fileInfoMap.get(fileId);
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
index 2fc43fc..4f84466 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
@@ -15,7 +15,6 @@
 package edu.uci.ics.hyracks.test.support;
 
 import java.nio.ByteBuffer;
-import java.util.UUID;
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
@@ -23,22 +22,21 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
 import edu.uci.ics.hyracks.api.resources.IDeallocatable;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
 
 public class TestJobletContext implements IHyracksJobletContext {
     private final INCApplicationContext appContext;
-    private UUID jobId;
-    private int attempt;
-    private ManagedWorkspaceFileFactory fileFactory;
+    private JobId jobId;
+    private WorkspaceFileFactory fileFactory;
 
-    public TestJobletContext(INCApplicationContext appContext, UUID jobId, int attempt) throws HyracksException {
+    public TestJobletContext(INCApplicationContext appContext, JobId jobId) throws HyracksException {
         this.appContext = appContext;
         this.jobId = jobId;
-        this.attempt = attempt;
-        fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) getIOManager());
+        fileFactory = new WorkspaceFileFactory(this, (IOManager) getIOManager());
     }
 
     @Override
@@ -57,13 +55,18 @@
     }
 
     @Override
-    public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
-        return fileFactory.createWorkspaceFile(prefix);
+    public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
+        return fileFactory.createManagedWorkspaceFile(prefix);
+    }
+
+    @Override
+    public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException {
+        return fileFactory.createUnmanagedWorkspaceFile(prefix);
     }
 
     @Override
     public ICounterContext getCounterContext() {
-        return new CounterContext(jobId + "." + attempt);
+        return new CounterContext(jobId.toString());
     }
 
     @Override
@@ -82,12 +85,7 @@
     }
 
     @Override
-    public UUID getJobId() {
+    public JobId getJobId() {
         return jobId;
     }
-
-    @Override
-    public int getAttempt() {
-        return attempt;
-    }
 }
\ No newline at end of file
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java
deleted file mode 100644
index 4d31326..0000000
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.test.support;
-
-import java.nio.ByteBuffer;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.resources.IDeallocatable;
-import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
-
-public class TestStageletContext implements IHyracksStageletContext {
-    private final IHyracksJobletContext jobletContext;
-    private UUID stageId;
-    private ManagedWorkspaceFileFactory fileFactory;
-
-    public TestStageletContext(IHyracksJobletContext jobletContext, UUID stageId) throws HyracksException {
-        this.jobletContext = jobletContext;
-        this.stageId = stageId;
-        fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) getIOManager());
-    }
-
-    @Override
-    public ByteBuffer allocateFrame() {
-        return jobletContext.allocateFrame();
-    }
-
-    @Override
-    public int getFrameSize() {
-        return jobletContext.getFrameSize();
-    }
-
-    @Override
-    public IIOManager getIOManager() {
-        return jobletContext.getIOManager();
-    }
-
-    @Override
-    public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
-        return fileFactory.createWorkspaceFile(prefix);
-    }
-
-    @Override
-    public IHyracksJobletContext getJobletContext() {
-        return jobletContext;
-    }
-
-    @Override
-    public UUID getStageId() {
-        return stageId;
-    }
-
-    @Override
-    public ICounterContext getCounterContext() {
-        return new CounterContext(jobletContext.getJobId() + "." + jobletContext.getAttempt() + "." + stageId);
-    }
-
-    @Override
-    public void registerDeallocatable(final IDeallocatable deallocatable) {
-        Runtime.getRuntime().addShutdownHook(new Thread() {
-            @Override
-            public void run() {
-                deallocatable.deallocate();
-            }
-        });
-    }
-}
\ No newline at end of file
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
index 2325db7..c7bd09d 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.test.support;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
 import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
@@ -45,7 +45,7 @@
         treeIndexRegistry = null;
     }
 
-    public synchronized static IBufferCache getBufferCache(IHyracksStageletContext ctx) {
+    public synchronized static IBufferCache getBufferCache(IHyracksTaskContext ctx) {
         if (bufferCache == null) {
             ICacheMemoryAllocator allocator = new HeapBufferAllocator();
             IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
@@ -56,16 +56,16 @@
         return bufferCache;
     }
 
-    public synchronized static IFileMapProvider getFileMapProvider(IHyracksStageletContext ctx) {
+    public synchronized static IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
         if (fileMapProvider == null) {
             fileMapProvider = new TransientFileMapManager();
         }
         return fileMapProvider;
     }
 
-    public synchronized static IndexRegistry<ITreeIndex> getTreeIndexRegistry(IHyracksStageletContext ctx) {
+    public synchronized static IndexRegistry<ITreeIndex> getTreeIndexRegistry(IHyracksTaskContext ctx) {
         if (treeIndexRegistry == null) {
-        	treeIndexRegistry = new IndexRegistry<ITreeIndex>();
+            treeIndexRegistry = new IndexRegistry<ITreeIndex>();
         }
         return treeIndexRegistry;
     }
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
index e923cdd..4059ef0 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.test.support;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -23,12 +23,12 @@
     private static final long serialVersionUID = 1L;
 
     @Override
-    public IBufferCache getBufferCache(IHyracksStageletContext ctx) {
+    public IBufferCache getBufferCache(IHyracksTaskContext ctx) {
         return TestStorageManagerComponentHolder.getBufferCache(ctx);
     }
 
     @Override
-    public IFileMapProvider getFileMapProvider(IHyracksStageletContext ctx) {
+    public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
         return TestStorageManagerComponentHolder.getFileMapProvider(ctx);
     }
 }
\ No newline at end of file
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
new file mode 100644
index 0000000..83b608f
--- /dev/null
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.test.support;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
+import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
+
+public class TestTaskContext implements IHyracksTaskContext {
+    private final IHyracksJobletContext jobletContext;
+    private final TaskAttemptId taskId;
+    private WorkspaceFileFactory fileFactory;
+
+    public TestTaskContext(IHyracksJobletContext jobletContext, TaskAttemptId taskId) throws HyracksException {
+        this.jobletContext = jobletContext;
+        this.taskId = taskId;
+        fileFactory = new WorkspaceFileFactory(this, (IOManager) getIOManager());
+    }
+
+    @Override
+    public ByteBuffer allocateFrame() {
+        return jobletContext.allocateFrame();
+    }
+
+    @Override
+    public int getFrameSize() {
+        return jobletContext.getFrameSize();
+    }
+
+    @Override
+    public IIOManager getIOManager() {
+        return jobletContext.getIOManager();
+    }
+
+    @Override
+    public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
+        return fileFactory.createManagedWorkspaceFile(prefix);
+    }
+
+    @Override
+    public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException {
+        return fileFactory.createUnmanagedWorkspaceFile(prefix);
+    }
+
+    @Override
+    public IHyracksJobletContext getJobletContext() {
+        return jobletContext;
+    }
+
+    @Override
+    public ICounterContext getCounterContext() {
+        return new CounterContext(jobletContext.getJobId() + "." + taskId);
+    }
+
+    @Override
+    public void registerDeallocatable(final IDeallocatable deallocatable) {
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                deallocatable.deallocate();
+            }
+        });
+    }
+
+    @Override
+    public TaskAttemptId getTaskAttemptId() {
+        return taskId;
+    }
+
+    @Override
+    public Object lookupGlobalVariable(ActivityId producerActivity, int partition, String varName)
+            throws HyracksDataException {
+        return null;
+    }
+
+    @Override
+    public Object lookupLocalVariable(ActivityId producerActivity, int partition, String varName)
+            throws HyracksDataException {
+        return null;
+    }
+
+    @Override
+    public void setVariable(String name, Object value) throws HyracksDataException {
+
+    }
+}
\ No newline at end of file
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java
index 630f47f..9925fe8 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java
@@ -14,16 +14,16 @@
  */
 package edu.uci.ics.hyracks.test.support;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
 
 public class TestTreeIndexRegistryProvider implements IIndexRegistryProvider<ITreeIndex> {
     private static final long serialVersionUID = 1L;
-    
-	@Override
-	public IndexRegistry<ITreeIndex> getRegistry(IHyracksStageletContext ctx) {
-		return TestStorageManagerComponentHolder.getTreeIndexRegistry(ctx);
-	}
+
+    @Override
+    public IndexRegistry<ITreeIndex> getRegistry(IHyracksTaskContext ctx) {
+        return TestStorageManagerComponentHolder.getTreeIndexRegistry(ctx);
+    }
 }
\ No newline at end of file
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestUtils.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestUtils.java
index b17f28c..5d488c1 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestUtils.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestUtils.java
@@ -14,22 +14,26 @@
  */
 package edu.uci.ics.hyracks.test.support;
 
-import java.util.UUID;
-
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
 
 public class TestUtils {
-    public static IHyracksStageletContext create(int frameSize) {
+    public static IHyracksTaskContext create(int frameSize) {
         try {
             IHyracksRootContext rootCtx = new TestRootContext(frameSize);
             INCApplicationContext appCtx = new TestNCApplicationContext(rootCtx, null);
-            IHyracksJobletContext jobletCtx = new TestJobletContext(appCtx, UUID.randomUUID(), 0);
-            IHyracksStageletContext stageletCtx = new TestStageletContext(jobletCtx, UUID.randomUUID());
-            return stageletCtx;
+            IHyracksJobletContext jobletCtx = new TestJobletContext(appCtx, new JobId(0));
+            TaskAttemptId tid = new TaskAttemptId(new TaskId(new ActivityId(new OperatorDescriptorId(0), 0), 0), 0);
+            IHyracksTaskContext taskCtx = new TestTaskContext(jobletCtx, tid);
+            return taskCtx;
         } catch (HyracksException e) {
             throw new RuntimeException(e);
         }
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
index 2b2bf55..4402e22 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
@@ -24,7 +24,7 @@
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -59,9 +59,9 @@
 	private static final int NUM_PAGES = 40;
 	private static final int MAX_OPEN_FILES = 10;
 	private static final int HYRACKS_FRAME_SIZE = 128;
-	private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+	private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
 		
-	private ITupleReference createTuple(IHyracksStageletContext ctx, int f0,
+    private ITupleReference createTuple(IHyracksTaskContext ctx, int f0,
 			int f1, int f2, boolean print) throws HyracksDataException {
 		if (print)
 		    LOGGER.info("CREATING: " + f0 + " " + f1 + " " + f2);		
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
index 44cf18b..3ef0cc2 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
@@ -8,7 +8,7 @@
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -47,127 +47,116 @@
 
 public class BTreeStatsTest extends AbstractBTreeTest {
 
-	// private static final int PAGE_SIZE = 256;
-	// private static final int NUM_PAGES = 10;
-	// private static final int PAGE_SIZE = 32768;
+    // private static final int PAGE_SIZE = 256;
+    // private static final int NUM_PAGES = 10;
+    // private static final int PAGE_SIZE = 32768;
     private static final int PAGE_SIZE = 4096;
     private static final int NUM_PAGES = 1000;
     private static final int MAX_OPEN_FILES = 10;
-	private static final int HYRACKS_FRAME_SIZE = 128;
-	private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
-	
-	@Test
-	public void test01() throws Exception {
+    private static final int HYRACKS_FRAME_SIZE = 128;
+    private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
 
-		TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
-		IBufferCache bufferCache = TestStorageManagerComponentHolder
-				.getBufferCache(ctx);
-		IFileMapProvider fmp = TestStorageManagerComponentHolder
-				.getFileMapProvider(ctx);
-		FileReference file = new FileReference(new File(fileName));
-		bufferCache.createFile(file);
-		int fileId = fmp.lookupFileId(file);
-		bufferCache.openFile(fileId);
+    @Test
+    public void test01() throws Exception {
 
-		// declare fields
-		int fieldCount = 2;
-		ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-		typeTraits[0] = new TypeTrait(4);
-		typeTraits[1] = new TypeTrait(4);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(fileName));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
+        bufferCache.openFile(fileId);
 
-		// declare keys
-		int keyFieldCount = 1;
-		IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-		cmps[0] = IntegerBinaryComparatorFactory.INSTANCE
-				.createBinaryComparator();
+        // declare fields
+        int fieldCount = 2;
+        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+        typeTraits[0] = new TypeTrait(4);
+        typeTraits[1] = new TypeTrait(4);
 
-		MultiComparator cmp = new MultiComparator(typeTraits, cmps);
+        // declare keys
+        int keyFieldCount = 1;
+        IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+        cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
 
-		TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
-				typeTraits);
-		ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
-				tupleWriterFactory);
-		ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(
-				tupleWriterFactory);
-		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        MultiComparator cmp = new MultiComparator(typeTraits, cmps);
 
-		IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
-		IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+        TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
-		IFreePageManager freePageManager = new LinkedListFreePageManager(
-				bufferCache, fileId, 0, metaFrameFactory);
+        IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
+        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
-		BTree btree = new BTree(bufferCache, freePageManager,
-				interiorFrameFactory, leafFrameFactory, cmp);
-		btree.create(fileId, leafFrame, metaFrame);
-		btree.open(fileId);
+        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
 
-		Random rnd = new Random();
-		rnd.setSeed(50);
+        BTree btree = new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmp);
+        btree.create(fileId, leafFrame, metaFrame);
+        btree.open(fileId);
 
-		long start = System.currentTimeMillis();
+        Random rnd = new Random();
+        rnd.setSeed(50);
 
-		LOGGER.info("INSERTING INTO TREE");
+        long start = System.currentTimeMillis();
 
-		ByteBuffer frame = ctx.allocateFrame();
-		FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-		ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
-		DataOutput dos = tb.getDataOutput();
+        LOGGER.info("INSERTING INTO TREE");
 
-		ISerializerDeserializer[] recDescSers = {
-				IntegerSerializerDeserializer.INSTANCE,
-				IntegerSerializerDeserializer.INSTANCE };
-		RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-		IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx
-				.getFrameSize(), recDesc);
-		accessor.reset(frame);
-		FrameTupleReference tuple = new FrameTupleReference();
+        ByteBuffer frame = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
+        DataOutput dos = tb.getDataOutput();
 
-		BTreeOpContext insertOpCtx = btree.createOpContext(
-				IndexOp.INSERT, leafFrame, interiorFrame, metaFrame);
+        ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE };
+        RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+        accessor.reset(frame);
+        FrameTupleReference tuple = new FrameTupleReference();
 
-		// 10000
-		for (int i = 0; i < 100000; i++) {
+        BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT, leafFrame, interiorFrame, metaFrame);
 
-			int f0 = rnd.nextInt() % 100000;
-			int f1 = 5;
+        // 10000
+        for (int i = 0; i < 100000; i++) {
 
-			tb.reset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(f0, dos);
-			tb.addFieldEndOffset();
-			IntegerSerializerDeserializer.INSTANCE.serialize(f1, dos);
-			tb.addFieldEndOffset();
+            int f0 = rnd.nextInt() % 100000;
+            int f1 = 5;
 
-			appender.reset(frame, true);
-			appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb
-					.getSize());
+            tb.reset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(f0, dos);
+            tb.addFieldEndOffset();
+            IntegerSerializerDeserializer.INSTANCE.serialize(f1, dos);
+            tb.addFieldEndOffset();
 
-			tuple.reset(accessor, 0);
-			
-			if (i % 10000 == 0) {
-				long end = System.currentTimeMillis();
-				LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " "
-						+ (end - start));
-			}
+            appender.reset(frame, true);
+            appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
 
-			try {
-				btree.insert(tuple, insertOpCtx);
-			} catch (TreeIndexException e) {
-			} catch (Exception e) {
-				e.printStackTrace();
-			}			
-		}
-		
-		TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId, btree.getRootPageId());		
-		TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
-		LOGGER.info(stats.toString());
+            tuple.reset(accessor, 0);
 
-		TreeIndexBufferCacheWarmup bufferCacheWarmup = new TreeIndexBufferCacheWarmup(bufferCache, freePageManager, fileId);
-		bufferCacheWarmup.warmup(leafFrame, metaFrame, new int[] {1, 2}, new int[] {2, 5});
-		
-		btree.close();
-		bufferCache.closeFile(fileId);
-		bufferCache.close();
-	}
+            if (i % 10000 == 0) {
+                long end = System.currentTimeMillis();
+                LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+            }
+
+            try {
+                btree.insert(tuple, insertOpCtx);
+            } catch (TreeIndexException e) {
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+
+        TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
+                btree.getRootPageId());
+        TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
+        LOGGER.info(stats.toString());
+
+        TreeIndexBufferCacheWarmup bufferCacheWarmup = new TreeIndexBufferCacheWarmup(bufferCache, freePageManager,
+                fileId);
+        bufferCacheWarmup.warmup(leafFrame, metaFrame, new int[] { 1, 2 }, new int[] { 2, 5 });
+
+        btree.close();
+        bufferCache.closeFile(fileId);
+        bufferCache.close();
+    }
 }
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
index 6fa5d43..a9debd5 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
@@ -23,7 +23,7 @@
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -73,7 +73,7 @@
     private static final int NUM_PAGES = 10;
     private static final int MAX_OPEN_FILES = 10;
     private static final int HYRACKS_FRAME_SIZE = 128;
-    private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+    private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
 
     // FIXED-LENGTH KEY TEST
     // create a B-tree with one fixed-length "key" field and one fixed-length
@@ -175,8 +175,8 @@
 
         int maxPage = btree.getFreePageManager().getMaxPage(metaFrame);
         LOGGER.info("MAXPAGE: " + maxPage);
-        LOGGER.info(btree.printStats());        
-        
+        LOGGER.info(btree.printStats());
+
         long end = System.currentTimeMillis();
         long duration = end - start;
         LOGGER.info("DURATION: " + duration);
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
index 1a3b219..7dada06 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
@@ -30,7 +30,7 @@
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -92,7 +92,7 @@
 	IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
 	ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
-	IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+    IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
 	ByteBuffer frame = ctx.allocateFrame();
 	FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
 
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
index 1a510e6..39bedac 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
@@ -22,10 +22,9 @@
 
 import org.junit.Test;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.storage.am.btree.AbstractBTreeTest;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
@@ -35,238 +34,225 @@
 import edu.uci.ics.hyracks.test.support.TestUtils;
 
 public class StorageManagerTest extends AbstractBTreeTest {
-	private static final int PAGE_SIZE = 256;
-	private static final int NUM_PAGES = 10;
-	private static final int MAX_OPEN_FILES = 10;
-	private static final int HYRACKS_FRAME_SIZE = 128;
-	private IHyracksStageletContext ctx = TestUtils.create(32768);
+    private static final int PAGE_SIZE = 256;
+    private static final int NUM_PAGES = 10;
+    private static final int MAX_OPEN_FILES = 10;
+    private static final int HYRACKS_FRAME_SIZE = 128;
+    private IHyracksTaskContext ctx = TestUtils.create(32768);
 
-	public class PinnedLatchedPage {
-		public final ICachedPage page;
-		public final LatchType latch;
-		public final int pageId;
+    public class PinnedLatchedPage {
+        public final ICachedPage page;
+        public final LatchType latch;
+        public final int pageId;
 
-		public PinnedLatchedPage(ICachedPage page, int pageId, LatchType latch) {
-			this.page = page;
-			this.pageId = pageId;
-			this.latch = latch;
-		}
-	}
+        public PinnedLatchedPage(ICachedPage page, int pageId, LatchType latch) {
+            this.page = page;
+            this.pageId = pageId;
+            this.latch = latch;
+        }
+    }
 
-	public enum FileAccessType {
-		FTA_READONLY, FTA_WRITEONLY, FTA_MIXED, FTA_UNLATCHED
-	}
+    public enum FileAccessType {
+        FTA_READONLY, FTA_WRITEONLY, FTA_MIXED, FTA_UNLATCHED
+    }
 
-	public class FileAccessWorker implements Runnable {
-		private int workerId;
-		private final IBufferCache bufferCache;
-		private final int maxPages;
-		private final int fileId;
-		private final long thinkTime;
-		private final int maxLoopCount;
-		private final int maxPinnedPages;
-		private final int closeFileChance;
-		private final FileAccessType fta;
-		private int loopCount = 0;
-		private boolean fileIsOpen = false;
-		private Random rnd = new Random(50);
-		private List<PinnedLatchedPage> pinnedPages = new LinkedList<PinnedLatchedPage>();
+    public class FileAccessWorker implements Runnable {
+        private int workerId;
+        private final IBufferCache bufferCache;
+        private final int maxPages;
+        private final int fileId;
+        private final long thinkTime;
+        private final int maxLoopCount;
+        private final int maxPinnedPages;
+        private final int closeFileChance;
+        private final FileAccessType fta;
+        private int loopCount = 0;
+        private boolean fileIsOpen = false;
+        private Random rnd = new Random(50);
+        private List<PinnedLatchedPage> pinnedPages = new LinkedList<PinnedLatchedPage>();
 
-		public FileAccessWorker(int workerId, IBufferCache bufferCache,
-				FileAccessType fta, int fileId, int maxPages,
-				int maxPinnedPages, int maxLoopCount, int closeFileChance,
-				long thinkTime) {
-			this.bufferCache = bufferCache;
-			this.fileId = fileId;
-			this.maxPages = maxPages;
-			this.maxLoopCount = maxLoopCount;
-			this.maxPinnedPages = maxPinnedPages;
-			this.thinkTime = thinkTime;
-			this.closeFileChance = closeFileChance;
-			this.workerId = workerId;
-			this.fta = fta;
-		}
+        public FileAccessWorker(int workerId, IBufferCache bufferCache, FileAccessType fta, int fileId, int maxPages,
+                int maxPinnedPages, int maxLoopCount, int closeFileChance, long thinkTime) {
+            this.bufferCache = bufferCache;
+            this.fileId = fileId;
+            this.maxPages = maxPages;
+            this.maxLoopCount = maxLoopCount;
+            this.maxPinnedPages = maxPinnedPages;
+            this.thinkTime = thinkTime;
+            this.closeFileChance = closeFileChance;
+            this.workerId = workerId;
+            this.fta = fta;
+        }
 
-		private void pinRandomPage() {
-			int pageId = Math.abs(rnd.nextInt() % maxPages);
+        private void pinRandomPage() {
+            int pageId = Math.abs(rnd.nextInt() % maxPages);
 
-			LOGGER.info(workerId + " PINNING PAGE: " + pageId);
+            LOGGER.info(workerId + " PINNING PAGE: " + pageId);
 
-			try {
-				ICachedPage page = bufferCache.pin(BufferedFileHandle
-						.getDiskPageId(fileId, pageId), false);
-				LatchType latch = null;
+            try {
+                ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+                LatchType latch = null;
 
-				switch (fta) {
+                switch (fta) {
 
-				case FTA_UNLATCHED: {
-					latch = null;
-				}
-					break;
+                    case FTA_UNLATCHED: {
+                        latch = null;
+                    }
+                        break;
 
-				case FTA_READONLY: {
-				    LOGGER.info(workerId + " S LATCHING: " + pageId);
-					page.acquireReadLatch();
-					latch = LatchType.LATCH_S;
-				}
-					break;
+                    case FTA_READONLY: {
+                        LOGGER.info(workerId + " S LATCHING: " + pageId);
+                        page.acquireReadLatch();
+                        latch = LatchType.LATCH_S;
+                    }
+                        break;
 
-				case FTA_WRITEONLY: {
-				    LOGGER.info(workerId + " X LATCHING: " + pageId);
-					page.acquireWriteLatch();
-					latch = LatchType.LATCH_X;
-				}
-					break;
+                    case FTA_WRITEONLY: {
+                        LOGGER.info(workerId + " X LATCHING: " + pageId);
+                        page.acquireWriteLatch();
+                        latch = LatchType.LATCH_X;
+                    }
+                        break;
 
-				case FTA_MIXED: {
-					if (rnd.nextInt() % 2 == 0) {
-					    LOGGER.info(workerId + " S LATCHING: " + pageId);
-						page.acquireReadLatch();
-						latch = LatchType.LATCH_S;
-					} else {
-					    LOGGER.info(workerId + " X LATCHING: " + pageId);
-						page.acquireWriteLatch();
-						latch = LatchType.LATCH_X;
-					}
-				}
-					break;
+                    case FTA_MIXED: {
+                        if (rnd.nextInt() % 2 == 0) {
+                            LOGGER.info(workerId + " S LATCHING: " + pageId);
+                            page.acquireReadLatch();
+                            latch = LatchType.LATCH_S;
+                        } else {
+                            LOGGER.info(workerId + " X LATCHING: " + pageId);
+                            page.acquireWriteLatch();
+                            latch = LatchType.LATCH_X;
+                        }
+                    }
+                        break;
 
-				}
+                }
 
-				PinnedLatchedPage plPage = new PinnedLatchedPage(page, pageId,
-						latch);
-				pinnedPages.add(plPage);
-			} catch (HyracksDataException e) {
-				e.printStackTrace();
-			}
-		}
+                PinnedLatchedPage plPage = new PinnedLatchedPage(page, pageId, latch);
+                pinnedPages.add(plPage);
+            } catch (HyracksDataException e) {
+                e.printStackTrace();
+            }
+        }
 
-		private void unpinRandomPage() {
-			int index = Math.abs(rnd.nextInt() % pinnedPages.size());
-			try {
-				PinnedLatchedPage plPage = pinnedPages.get(index);
+        private void unpinRandomPage() {
+            int index = Math.abs(rnd.nextInt() % pinnedPages.size());
+            try {
+                PinnedLatchedPage plPage = pinnedPages.get(index);
 
-				if (plPage.latch != null) {
-					if (plPage.latch == LatchType.LATCH_S) {
-					    LOGGER.info(workerId + " S UNLATCHING: "
-								+ plPage.pageId);
-						plPage.page.releaseReadLatch();
-					} else {
-					    LOGGER.info(workerId + " X UNLATCHING: "
-								+ plPage.pageId);
-						plPage.page.releaseWriteLatch();
-					}
-				}
-				LOGGER.info(workerId + " UNPINNING PAGE: "
-						+ plPage.pageId);
+                if (plPage.latch != null) {
+                    if (plPage.latch == LatchType.LATCH_S) {
+                        LOGGER.info(workerId + " S UNLATCHING: " + plPage.pageId);
+                        plPage.page.releaseReadLatch();
+                    } else {
+                        LOGGER.info(workerId + " X UNLATCHING: " + plPage.pageId);
+                        plPage.page.releaseWriteLatch();
+                    }
+                }
+                LOGGER.info(workerId + " UNPINNING PAGE: " + plPage.pageId);
 
-				bufferCache.unpin(plPage.page);
-				pinnedPages.remove(index);
-			} catch (HyracksDataException e) {
-				e.printStackTrace();
-			}
-		}
+                bufferCache.unpin(plPage.page);
+                pinnedPages.remove(index);
+            } catch (HyracksDataException e) {
+                e.printStackTrace();
+            }
+        }
 
-		private void openFile() {
-		    LOGGER.info(workerId + " OPENING FILE: " + fileId);
-			try {
-				bufferCache.openFile(fileId);
-				fileIsOpen = true;
-			} catch (HyracksDataException e) {
-				e.printStackTrace();
-			}
-		}
+        private void openFile() {
+            LOGGER.info(workerId + " OPENING FILE: " + fileId);
+            try {
+                bufferCache.openFile(fileId);
+                fileIsOpen = true;
+            } catch (HyracksDataException e) {
+                e.printStackTrace();
+            }
+        }
 
-		private void closeFile() {
-		    LOGGER.info(workerId + " CLOSING FILE: " + fileId);
-			try {
-				bufferCache.closeFile(fileId);
-				fileIsOpen = false;
-			} catch (HyracksDataException e) {
-				e.printStackTrace();
-			}
-		}
+        private void closeFile() {
+            LOGGER.info(workerId + " CLOSING FILE: " + fileId);
+            try {
+                bufferCache.closeFile(fileId);
+                fileIsOpen = false;
+            } catch (HyracksDataException e) {
+                e.printStackTrace();
+            }
+        }
 
-		@Override
-		public void run() {
+        @Override
+        public void run() {
 
-			openFile();
+            openFile();
 
-			while (loopCount < maxLoopCount) {
-				loopCount++;
+            while (loopCount < maxLoopCount) {
+                loopCount++;
 
-				LOGGER.info(workerId + " LOOP: " + loopCount + "/"
-						+ maxLoopCount);
+                LOGGER.info(workerId + " LOOP: " + loopCount + "/" + maxLoopCount);
 
-				if (fileIsOpen) {
+                if (fileIsOpen) {
 
-					// pin some pages
-					int pagesToPin = Math.abs(rnd.nextInt())
-							% (maxPinnedPages - pinnedPages.size());
-					for (int i = 0; i < pagesToPin; i++) {
-						pinRandomPage();
-					}
+                    // pin some pages
+                    int pagesToPin = Math.abs(rnd.nextInt()) % (maxPinnedPages - pinnedPages.size());
+                    for (int i = 0; i < pagesToPin; i++) {
+                        pinRandomPage();
+                    }
 
-					// do some thinking
-					try {
-						Thread.sleep(thinkTime);
-					} catch (InterruptedException e) {
-						e.printStackTrace();
-					}
+                    // do some thinking
+                    try {
+                        Thread.sleep(thinkTime);
+                    } catch (InterruptedException e) {
+                        e.printStackTrace();
+                    }
 
-					// unpin some pages
-					if (!pinnedPages.isEmpty()) {
-						int pagesToUnpin = Math.abs(rnd.nextInt())
-								% pinnedPages.size();
-						for (int i = 0; i < pagesToUnpin; i++) {
-							unpinRandomPage();
-						}
-					}
+                    // unpin some pages
+                    if (!pinnedPages.isEmpty()) {
+                        int pagesToUnpin = Math.abs(rnd.nextInt()) % pinnedPages.size();
+                        for (int i = 0; i < pagesToUnpin; i++) {
+                            unpinRandomPage();
+                        }
+                    }
 
-					// possibly close file
-					int closeFileCheck = Math.abs(rnd.nextInt())
-							% closeFileChance;
-					if (pinnedPages.isEmpty() || closeFileCheck == 0) {
-						int numPinnedPages = pinnedPages.size();
-						for (int i = 0; i < numPinnedPages; i++) {
-							unpinRandomPage();
-						}
-						closeFile();
-					}
-				} else {
-					openFile();
-				}
-			}
+                    // possibly close file
+                    int closeFileCheck = Math.abs(rnd.nextInt()) % closeFileChance;
+                    if (pinnedPages.isEmpty() || closeFileCheck == 0) {
+                        int numPinnedPages = pinnedPages.size();
+                        for (int i = 0; i < numPinnedPages; i++) {
+                            unpinRandomPage();
+                        }
+                        closeFile();
+                    }
+                } else {
+                    openFile();
+                }
+            }
 
-			if (fileIsOpen) {
-				int numPinnedPages = pinnedPages.size();
-				for (int i = 0; i < numPinnedPages; i++) {
-					unpinRandomPage();
-				}
-				closeFile();
-			}
-		}
-	}
+            if (fileIsOpen) {
+                int numPinnedPages = pinnedPages.size();
+                for (int i = 0; i < numPinnedPages; i++) {
+                    unpinRandomPage();
+                }
+                closeFile();
+            }
+        }
+    }
 
-	@Test
-	public void oneThreadOneFileTest() throws Exception {
-		TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
-		IBufferCache bufferCache = TestStorageManagerComponentHolder
-				.getBufferCache(ctx);
-		IFileMapProvider fmp = TestStorageManagerComponentHolder
-				.getFileMapProvider(ctx);
-		FileReference file = new FileReference(new File(fileName));
-		bufferCache.createFile(file);
-		int fileId = fmp.lookupFileId(file);
-		bufferCache.openFile(fileId);
+    @Test
+    public void oneThreadOneFileTest() throws Exception {
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        FileReference file = new FileReference(new File(fileName));
+        bufferCache.createFile(file);
+        int fileId = fmp.lookupFileId(file);
+        bufferCache.openFile(fileId);
 
-		Thread worker = new Thread(new FileAccessWorker(0, bufferCache,
-				FileAccessType.FTA_UNLATCHED, fileId, 10, 10, 100, 10, 0));
+        Thread worker = new Thread(new FileAccessWorker(0, bufferCache, FileAccessType.FTA_UNLATCHED, fileId, 10, 10,
+                100, 10, 0));
 
-		worker.start();
+        worker.start();
 
-		worker.join();
+        worker.join();
 
-		bufferCache.close();
-	}
+        bufferCache.close();
+    }
 }
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
index b42001d..ea97511 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
@@ -10,7 +10,7 @@
 import org.junit.Before;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -49,176 +49,154 @@
 import edu.uci.ics.hyracks.test.support.TestUtils;
 
 public abstract class AbstractInvIndexSearchTest extends AbstractInvIndexTest {
-	protected final int PAGE_SIZE = 32768;
-	protected final int NUM_PAGES = 100;
-	protected final int MAX_OPEN_FILES = 10;
-	protected final int HYRACKS_FRAME_SIZE = 32768;
-	protected IHyracksStageletContext stageletCtx = TestUtils
-			.create(HYRACKS_FRAME_SIZE);
+    protected final int PAGE_SIZE = 32768;
+    protected final int NUM_PAGES = 100;
+    protected final int MAX_OPEN_FILES = 10;
+    protected final int HYRACKS_FRAME_SIZE = 32768;
+    protected IHyracksTaskContext taskCtx = TestUtils.create(HYRACKS_FRAME_SIZE);
 
-	protected IBufferCache bufferCache;
-	protected IFileMapProvider fmp;
+    protected IBufferCache bufferCache;
+    protected IFileMapProvider fmp;
 
-	// --- BTREE ---
+    // --- BTREE ---
 
-	// create file refs
-	protected FileReference btreeFile = new FileReference(new File(
-			btreeFileName));
-	protected int btreeFileId;
+    // create file refs
+    protected FileReference btreeFile = new FileReference(new File(btreeFileName));
+    protected int btreeFileId;
 
-	// declare btree fields
-	protected int fieldCount = 5;
-	protected ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+    // declare btree fields
+    protected int fieldCount = 5;
+    protected ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
 
-	// declare btree keys
-	protected int btreeKeyFieldCount = 1;
-	protected IBinaryComparator[] btreeBinCmps = new IBinaryComparator[btreeKeyFieldCount];
-	protected MultiComparator btreeCmp = new MultiComparator(typeTraits,
-			btreeBinCmps);
+    // declare btree keys
+    protected int btreeKeyFieldCount = 1;
+    protected IBinaryComparator[] btreeBinCmps = new IBinaryComparator[btreeKeyFieldCount];
+    protected MultiComparator btreeCmp = new MultiComparator(typeTraits, btreeBinCmps);
 
-	// btree frame factories
-	protected TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
-			typeTraits);
-	protected ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
-			tupleWriterFactory);
-	protected ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(
-			tupleWriterFactory);
-	protected ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+    // btree frame factories
+    protected TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+    protected ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+    protected ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+    protected ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
-	// btree frames
-	protected ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
-	protected ITreeIndexMetaDataFrame metaFrame = metaFrameFactory
-			.createFrame();
+    // btree frames
+    protected ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
+    protected ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
-	protected IFreePageManager freePageManager;
+    protected IFreePageManager freePageManager;
 
-	protected BTree btree;
+    protected BTree btree;
 
-	// --- INVERTED INDEX ---
+    // --- INVERTED INDEX ---
 
-	protected FileReference invListsFile = new FileReference(new File(
-			invListsFileName));
-	protected int invListsFileId;
+    protected FileReference invListsFile = new FileReference(new File(invListsFileName));
+    protected int invListsFileId;
 
-	protected int invListFields = 1;
-	protected ITypeTrait[] invListTypeTraits = new ITypeTrait[invListFields];
+    protected int invListFields = 1;
+    protected ITypeTrait[] invListTypeTraits = new ITypeTrait[invListFields];
 
-	protected int invListKeys = 1;
-	protected IBinaryComparator[] invListBinCmps = new IBinaryComparator[invListKeys];
-	protected MultiComparator invListCmp = new MultiComparator(
-			invListTypeTraits, invListBinCmps);
+    protected int invListKeys = 1;
+    protected IBinaryComparator[] invListBinCmps = new IBinaryComparator[invListKeys];
+    protected MultiComparator invListCmp = new MultiComparator(invListTypeTraits, invListBinCmps);
 
-	protected InvertedIndex invIndex;
+    protected InvertedIndex invIndex;
 
-	protected Random rnd = new Random();
+    protected Random rnd = new Random();
 
-	protected ByteBuffer frame = stageletCtx.allocateFrame();
-	protected FrameTupleAppender appender = new FrameTupleAppender(
-			stageletCtx.getFrameSize());
-	protected ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
-	protected DataOutput dos = tb.getDataOutput();
+    protected ByteBuffer frame = taskCtx.allocateFrame();
+    protected FrameTupleAppender appender = new FrameTupleAppender(taskCtx.getFrameSize());
+    protected ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
+    protected DataOutput dos = tb.getDataOutput();
 
-	protected ISerializerDeserializer[] insertSerde = {
-			UTF8StringSerializerDeserializer.INSTANCE,
-			IntegerSerializerDeserializer.INSTANCE };
-	protected RecordDescriptor insertRecDesc = new RecordDescriptor(insertSerde);
-	protected IFrameTupleAccessor accessor = new FrameTupleAccessor(
-			stageletCtx.getFrameSize(), insertRecDesc);
+    protected ISerializerDeserializer[] insertSerde = { UTF8StringSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE };
+    protected RecordDescriptor insertRecDesc = new RecordDescriptor(insertSerde);
+    protected IFrameTupleAccessor accessor = new FrameTupleAccessor(taskCtx.getFrameSize(), insertRecDesc);
 
-	protected FrameTupleReference tuple = new FrameTupleReference();
+    protected FrameTupleReference tuple = new FrameTupleReference();
 
-	protected ArrayList<ArrayList<Integer>> checkInvLists = new ArrayList<ArrayList<Integer>>();
+    protected ArrayList<ArrayList<Integer>> checkInvLists = new ArrayList<ArrayList<Integer>>();
 
-	protected int maxId = 1000000;
-	// protected int maxId = 1000;
-	protected int[] scanCountArray = new int[maxId];
-	protected ArrayList<Integer> expectedResults = new ArrayList<Integer>();
+    protected int maxId = 1000000;
+    // protected int maxId = 1000;
+    protected int[] scanCountArray = new int[maxId];
+    protected ArrayList<Integer> expectedResults = new ArrayList<Integer>();
 
-	protected ISerializerDeserializer[] querySerde = { UTF8StringSerializerDeserializer.INSTANCE };
-	protected RecordDescriptor queryRecDesc = new RecordDescriptor(querySerde);
+    protected ISerializerDeserializer[] querySerde = { UTF8StringSerializerDeserializer.INSTANCE };
+    protected RecordDescriptor queryRecDesc = new RecordDescriptor(querySerde);
 
-	protected FrameTupleAppender queryAppender = new FrameTupleAppender(
-			stageletCtx.getFrameSize());
-	protected ArrayTupleBuilder queryTb = new ArrayTupleBuilder(
-			querySerde.length);
-	protected DataOutput queryDos = queryTb.getDataOutput();
+    protected FrameTupleAppender queryAppender = new FrameTupleAppender(taskCtx.getFrameSize());
+    protected ArrayTupleBuilder queryTb = new ArrayTupleBuilder(querySerde.length);
+    protected DataOutput queryDos = queryTb.getDataOutput();
 
-	protected IFrameTupleAccessor queryAccessor = new FrameTupleAccessor(
-			stageletCtx.getFrameSize(), queryRecDesc);
-	protected FrameTupleReference queryTuple = new FrameTupleReference();
+    protected IFrameTupleAccessor queryAccessor = new FrameTupleAccessor(taskCtx.getFrameSize(), queryRecDesc);
+    protected FrameTupleReference queryTuple = new FrameTupleReference();
 
-	protected ITokenFactory tokenFactory;
-	protected IBinaryTokenizer tokenizer;
+    protected ITokenFactory tokenFactory;
+    protected IBinaryTokenizer tokenizer;
 
-	protected TOccurrenceSearcher searcher;
-	protected IInvertedIndexResultCursor resultCursor;
+    protected TOccurrenceSearcher searcher;
+    protected IInvertedIndexResultCursor resultCursor;
 
-	/**
-	 * Initialize members, generate data, and bulk load the inverted index.
-	 * 
-	 */
-	@Before
-	public void start() throws Exception {
-		TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
-				MAX_OPEN_FILES);
-		bufferCache = TestStorageManagerComponentHolder
-				.getBufferCache(stageletCtx);
-		fmp = TestStorageManagerComponentHolder.getFileMapProvider(stageletCtx);
+    /**
+     * Initialize members, generate data, and bulk load the inverted index.
+     */
+    @Before
+    public void start() throws Exception {
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+        bufferCache = TestStorageManagerComponentHolder.getBufferCache(taskCtx);
+        fmp = TestStorageManagerComponentHolder.getFileMapProvider(taskCtx);
 
-		// --- BTREE ---
+        // --- BTREE ---
 
-		bufferCache.createFile(btreeFile);
-		btreeFileId = fmp.lookupFileId(btreeFile);
-		bufferCache.openFile(btreeFileId);
+        bufferCache.createFile(btreeFile);
+        btreeFileId = fmp.lookupFileId(btreeFile);
+        bufferCache.openFile(btreeFileId);
 
-		// token (key)
-		typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		// startPageId
-		typeTraits[1] = new TypeTrait(4);
-		// endPageId
-		typeTraits[2] = new TypeTrait(4);
-		// startOff
-		typeTraits[3] = new TypeTrait(4);
-		// numElements
-		typeTraits[4] = new TypeTrait(4);
+        // token (key)
+        typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+        // startPageId
+        typeTraits[1] = new TypeTrait(4);
+        // endPageId
+        typeTraits[2] = new TypeTrait(4);
+        // startOff
+        typeTraits[3] = new TypeTrait(4);
+        // numElements
+        typeTraits[4] = new TypeTrait(4);
 
-		btreeBinCmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE
-				.createBinaryComparator();
+        btreeBinCmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
 
-		freePageManager = new LinkedListFreePageManager(bufferCache,
-				btreeFileId, 0, metaFrameFactory);
+        freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
 
-		btree = new BTree(bufferCache, freePageManager, interiorFrameFactory,
-				leafFrameFactory, btreeCmp);
-		btree.create(btreeFileId, leafFrame, metaFrame);
-		btree.open(btreeFileId);
+        btree = new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, btreeCmp);
+        btree.create(btreeFileId, leafFrame, metaFrame);
+        btree.open(btreeFileId);
 
-		// --- INVERTED INDEX ---
+        // --- INVERTED INDEX ---
 
-		bufferCache.createFile(invListsFile);
-		invListsFileId = fmp.lookupFileId(invListsFile);
-		bufferCache.openFile(invListsFileId);
+        bufferCache.createFile(invListsFile);
+        invListsFileId = fmp.lookupFileId(invListsFile);
+        bufferCache.openFile(invListsFileId);
 
-		invListTypeTraits[0] = new TypeTrait(4);
-		invListBinCmps[0] = IntegerBinaryComparatorFactory.INSTANCE
-				.createBinaryComparator();
+        invListTypeTraits[0] = new TypeTrait(4);
+        invListBinCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
 
-		invIndex = new InvertedIndex(bufferCache, btree, invListCmp);
-		invIndex.open(invListsFileId);
+        invIndex = new InvertedIndex(bufferCache, btree, invListCmp);
+        invIndex.open(invListsFileId);
 
-		rnd.setSeed(50);
+        rnd.setSeed(50);
 
-		accessor.reset(frame);
-		queryAccessor.reset(frame);
-	}
+        accessor.reset(frame);
+        queryAccessor.reset(frame);
+    }
 
-	@After
-	public void deinit() throws HyracksDataException {
-		AbstractInvIndexTest.tearDown();
-		btree.close();
-		invIndex.close();
-		bufferCache.closeFile(btreeFileId);
-		bufferCache.closeFile(invListsFileId);
-		bufferCache.close();
-	}
+    @After
+    public void deinit() throws HyracksDataException {
+        AbstractInvIndexTest.tearDown();
+        btree.close();
+        invIndex.close();
+        bufferCache.closeFile(btreeFileId);
+        bufferCache.closeFile(invListsFileId);
+        bufferCache.close();
+    }
 }
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
index fb0b7b9..bdd8c4a 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
@@ -28,7 +28,7 @@
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -74,272 +74,238 @@
 
 public class BulkLoadTest extends AbstractInvIndexTest {
 
-	private static final int PAGE_SIZE = 32768;
-	private static final int NUM_PAGES = 100;
-	private static final int MAX_OPEN_FILES = 10;
-	private static final int HYRACKS_FRAME_SIZE = 32768;
-	private IHyracksStageletContext stageletCtx = TestUtils
-			.create(HYRACKS_FRAME_SIZE);
+    private static final int PAGE_SIZE = 32768;
+    private static final int NUM_PAGES = 100;
+    private static final int MAX_OPEN_FILES = 10;
+    private static final int HYRACKS_FRAME_SIZE = 32768;
+    private IHyracksTaskContext stageletCtx = TestUtils.create(HYRACKS_FRAME_SIZE);
 
-	/**
-	 * This test generates a list of <word-token, id> pairs which are pre-sorted
-	 * on the token. Those pairs for the input to an inverted-index bulk load.
-	 * The contents of the inverted lists are verified against the generated
-	 * data.
-	 * 
-	 */
-	@Test
-	public void singleFieldPayloadTest() throws Exception {
+    /**
+     * This test generates a list of <word-token, id> pairs which are pre-sorted
+     * on the token. Those pairs for the input to an inverted-index bulk load.
+     * The contents of the inverted lists are verified against the generated
+     * data.
+     */
+    @Test
+    public void singleFieldPayloadTest() throws Exception {
 
-		TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
-				MAX_OPEN_FILES);
-		IBufferCache bufferCache = TestStorageManagerComponentHolder
-				.getBufferCache(stageletCtx);
-		IFileMapProvider fmp = TestStorageManagerComponentHolder
-				.getFileMapProvider(stageletCtx);
+        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(stageletCtx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(stageletCtx);
 
-		// create file refs
-		FileReference btreeFile = new FileReference(new File(btreeFileName));
-		bufferCache.createFile(btreeFile);
-		int btreeFileId = fmp.lookupFileId(btreeFile);
-		bufferCache.openFile(btreeFileId);
+        // create file refs
+        FileReference btreeFile = new FileReference(new File(btreeFileName));
+        bufferCache.createFile(btreeFile);
+        int btreeFileId = fmp.lookupFileId(btreeFile);
+        bufferCache.openFile(btreeFileId);
 
-		FileReference invListsFile = new FileReference(new File(
-				invListsFileName));
-		bufferCache.createFile(invListsFile);
-		int invListsFileId = fmp.lookupFileId(invListsFile);
-		bufferCache.openFile(invListsFileId);
+        FileReference invListsFile = new FileReference(new File(invListsFileName));
+        bufferCache.createFile(invListsFile);
+        int invListsFileId = fmp.lookupFileId(invListsFile);
+        bufferCache.openFile(invListsFileId);
 
-		// declare btree fields
-		int fieldCount = 5;
-		ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
-		// token (key)
-		typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		// startPageId
-		typeTraits[1] = new TypeTrait(4);
-		// endPageId
-		typeTraits[2] = new TypeTrait(4);
-		// startOff
-		typeTraits[3] = new TypeTrait(4);
-		// numElements
-		typeTraits[4] = new TypeTrait(4);
+        // declare btree fields
+        int fieldCount = 5;
+        ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+        // token (key)
+        typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+        // startPageId
+        typeTraits[1] = new TypeTrait(4);
+        // endPageId
+        typeTraits[2] = new TypeTrait(4);
+        // startOff
+        typeTraits[3] = new TypeTrait(4);
+        // numElements
+        typeTraits[4] = new TypeTrait(4);
 
-		// declare btree keys
-		int keyFieldCount = 1;
-		IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
-		cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE
-				.createBinaryComparator();
+        // declare btree keys
+        int keyFieldCount = 1;
+        IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+        cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
 
-		MultiComparator btreeCmp = new MultiComparator(typeTraits, cmps);
+        MultiComparator btreeCmp = new MultiComparator(typeTraits, cmps);
 
-		TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
-				typeTraits);
-		ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
-				tupleWriterFactory);
-		ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(
-				tupleWriterFactory);
-		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
-		ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
-		ITreeIndexFrame interiorFrame = interiorFrameFactory.createFrame();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+        ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
+        ITreeIndexFrame interiorFrame = interiorFrameFactory.createFrame();
+        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
-		IFreePageManager freePageManager = new LinkedListFreePageManager(
-				bufferCache, btreeFileId, 0, metaFrameFactory);
+        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
 
-		BTree btree = new BTree(bufferCache, freePageManager,
-				interiorFrameFactory, leafFrameFactory, btreeCmp);
-		btree.create(btreeFileId, leafFrame, metaFrame);
-		btree.open(btreeFileId);
+        BTree btree = new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, btreeCmp);
+        btree.create(btreeFileId, leafFrame, metaFrame);
+        btree.open(btreeFileId);
 
-		int invListFields = 1;
-		ITypeTrait[] invListTypeTraits = new ITypeTrait[invListFields];
-		invListTypeTraits[0] = new TypeTrait(4);
+        int invListFields = 1;
+        ITypeTrait[] invListTypeTraits = new ITypeTrait[invListFields];
+        invListTypeTraits[0] = new TypeTrait(4);
 
-		int invListKeys = 1;
-		IBinaryComparator[] invListBinCmps = new IBinaryComparator[invListKeys];
-		invListBinCmps[0] = IntegerBinaryComparatorFactory.INSTANCE
-				.createBinaryComparator();
+        int invListKeys = 1;
+        IBinaryComparator[] invListBinCmps = new IBinaryComparator[invListKeys];
+        invListBinCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
 
-		MultiComparator invListCmp = new MultiComparator(invListTypeTraits,
-				invListBinCmps);
+        MultiComparator invListCmp = new MultiComparator(invListTypeTraits, invListBinCmps);
 
-		InvertedIndex invIndex = new InvertedIndex(bufferCache, btree,
-				invListCmp);
-		invIndex.open(invListsFileId);
+        InvertedIndex invIndex = new InvertedIndex(bufferCache, btree, invListCmp);
+        invIndex.open(invListsFileId);
 
-		Random rnd = new Random();
-		rnd.setSeed(50);
+        Random rnd = new Random();
+        rnd.setSeed(50);
 
-		ByteBuffer frame = stageletCtx.allocateFrame();
-		FrameTupleAppender appender = new FrameTupleAppender(
-				stageletCtx.getFrameSize());
-		ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
-		DataOutput dos = tb.getDataOutput();
+        ByteBuffer frame = stageletCtx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(stageletCtx.getFrameSize());
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
+        DataOutput dos = tb.getDataOutput();
 
-		ISerializerDeserializer[] insertSerde = {
-				UTF8StringSerializerDeserializer.INSTANCE,
-				IntegerSerializerDeserializer.INSTANCE };
-		RecordDescriptor insertRecDesc = new RecordDescriptor(insertSerde);
-		IFrameTupleAccessor accessor = new FrameTupleAccessor(
-				stageletCtx.getFrameSize(), insertRecDesc);
-		accessor.reset(frame);
-		FrameTupleReference tuple = new FrameTupleReference();
+        ISerializerDeserializer[] insertSerde = { UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE };
+        RecordDescriptor insertRecDesc = new RecordDescriptor(insertSerde);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), insertRecDesc);
+        accessor.reset(frame);
+        FrameTupleReference tuple = new FrameTupleReference();
 
-		List<String> tokens = new ArrayList<String>();
-		tokens.add("compilers");
-		tokens.add("computer");
-		tokens.add("databases");
-		tokens.add("fast");
-		tokens.add("hyracks");
-		tokens.add("major");
-		tokens.add("science");
-		tokens.add("systems");
-		tokens.add("university");
+        List<String> tokens = new ArrayList<String>();
+        tokens.add("compilers");
+        tokens.add("computer");
+        tokens.add("databases");
+        tokens.add("fast");
+        tokens.add("hyracks");
+        tokens.add("major");
+        tokens.add("science");
+        tokens.add("systems");
+        tokens.add("university");
 
-		ArrayList<ArrayList<Integer>> checkListElements = new ArrayList<ArrayList<Integer>>();
-		for (int i = 0; i < tokens.size(); i++) {
-			checkListElements.add(new ArrayList<Integer>());
-		}
+        ArrayList<ArrayList<Integer>> checkListElements = new ArrayList<ArrayList<Integer>>();
+        for (int i = 0; i < tokens.size(); i++) {
+            checkListElements.add(new ArrayList<Integer>());
+        }
 
-		int maxId = 1000000;
-		int addProb = 0;
-		int addProbStep = 10;
+        int maxId = 1000000;
+        int addProb = 0;
+        int addProbStep = 10;
 
-		IInvertedListBuilder invListBuilder = new FixedSizeElementInvertedListBuilder(
-				invListTypeTraits);
-		InvertedIndex.BulkLoadContext ctx = invIndex.beginBulkLoad(
-				invListBuilder, HYRACKS_FRAME_SIZE, BTree.DEFAULT_FILL_FACTOR);
+        IInvertedListBuilder invListBuilder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
+        InvertedIndex.BulkLoadContext ctx = invIndex.beginBulkLoad(invListBuilder, HYRACKS_FRAME_SIZE,
+                BTree.DEFAULT_FILL_FACTOR);
 
-		int totalElements = 0;
-		for (int i = 0; i < tokens.size(); i++) {
+        int totalElements = 0;
+        for (int i = 0; i < tokens.size(); i++) {
 
-			addProb += addProbStep * (i + 1);
-			for (int j = 0; j < maxId; j++) {
-				if ((Math.abs(rnd.nextInt()) % addProb) == 0) {
+            addProb += addProbStep * (i + 1);
+            for (int j = 0; j < maxId; j++) {
+                if ((Math.abs(rnd.nextInt()) % addProb) == 0) {
 
-					totalElements++;
+                    totalElements++;
 
-					tb.reset();
-					UTF8StringSerializerDeserializer.INSTANCE.serialize(
-							tokens.get(i), dos);
-					tb.addFieldEndOffset();
-					IntegerSerializerDeserializer.INSTANCE.serialize(j, dos);
-					tb.addFieldEndOffset();
+                    tb.reset();
+                    UTF8StringSerializerDeserializer.INSTANCE.serialize(tokens.get(i), dos);
+                    tb.addFieldEndOffset();
+                    IntegerSerializerDeserializer.INSTANCE.serialize(j, dos);
+                    tb.addFieldEndOffset();
 
-					checkListElements.get(i).add(j);
+                    checkListElements.get(i).add(j);
 
-					appender.reset(frame, true);
-					appender.append(tb.getFieldEndOffsets(), tb.getByteArray(),
-							0, tb.getSize());
+                    appender.reset(frame, true);
+                    appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
 
-					tuple.reset(accessor, 0);
+                    tuple.reset(accessor, 0);
 
-					try {
-						invIndex.bulkLoadAddTuple(ctx, tuple);
-					} catch (Exception e) {
-						e.printStackTrace();
-					}
-				}
-			}
-		}
-		invIndex.endBulkLoad(ctx);
+                    try {
+                        invIndex.bulkLoadAddTuple(ctx, tuple);
+                    } catch (Exception e) {
+                        e.printStackTrace();
+                    }
+                }
+            }
+        }
+        invIndex.endBulkLoad(ctx);
 
-		// ------- START VERIFICATION -----------
+        // ------- START VERIFICATION -----------
 
-		ITreeIndexCursor btreeCursor = new BTreeRangeSearchCursor(
-				(IBTreeLeafFrame) leafFrame);
-		FrameTupleReference searchKey = new FrameTupleReference();
-		RangePredicate btreePred = new RangePredicate(true, searchKey,
-				searchKey, true, true, btreeCmp, btreeCmp);
+        ITreeIndexCursor btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame);
+        FrameTupleReference searchKey = new FrameTupleReference();
+        RangePredicate btreePred = new RangePredicate(true, searchKey, searchKey, true, true, btreeCmp, btreeCmp);
 
-		IInvertedListCursor invListCursor = new FixedSizeElementInvertedListCursor(
-				bufferCache, invListsFileId, invListTypeTraits);
+        IInvertedListCursor invListCursor = new FixedSizeElementInvertedListCursor(bufferCache, invListsFileId,
+                invListTypeTraits);
 
-		ISerializerDeserializer[] tokenSerde = { UTF8StringSerializerDeserializer.INSTANCE };
-		RecordDescriptor tokenRecDesc = new RecordDescriptor(tokenSerde);
-		FrameTupleAppender tokenAppender = new FrameTupleAppender(
-				stageletCtx.getFrameSize());
-		ArrayTupleBuilder tokenTupleBuilder = new ArrayTupleBuilder(1);
-		DataOutput tokenDos = tokenTupleBuilder.getDataOutput();
-		IFrameTupleAccessor tokenAccessor = new FrameTupleAccessor(
-				stageletCtx.getFrameSize(), tokenRecDesc);
-		tokenAccessor.reset(frame);
+        ISerializerDeserializer[] tokenSerde = { UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor tokenRecDesc = new RecordDescriptor(tokenSerde);
+        FrameTupleAppender tokenAppender = new FrameTupleAppender(stageletCtx.getFrameSize());
+        ArrayTupleBuilder tokenTupleBuilder = new ArrayTupleBuilder(1);
+        DataOutput tokenDos = tokenTupleBuilder.getDataOutput();
+        IFrameTupleAccessor tokenAccessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), tokenRecDesc);
+        tokenAccessor.reset(frame);
 
-		BTreeOpContext btreeOpCtx = invIndex.getBTree().createOpContext(
-				IndexOp.SEARCH, leafFrame, interiorFrame, null);
+        BTreeOpContext btreeOpCtx = invIndex.getBTree().createOpContext(IndexOp.SEARCH, leafFrame, interiorFrame, null);
 
-		// verify created inverted lists one-by-one
-		for (int i = 0; i < tokens.size(); i++) {
+        // verify created inverted lists one-by-one
+        for (int i = 0; i < tokens.size(); i++) {
 
-			tokenTupleBuilder.reset();
-			UTF8StringSerializerDeserializer.INSTANCE.serialize(tokens.get(i),
-					tokenDos);
-			tokenTupleBuilder.addFieldEndOffset();
+            tokenTupleBuilder.reset();
+            UTF8StringSerializerDeserializer.INSTANCE.serialize(tokens.get(i), tokenDos);
+            tokenTupleBuilder.addFieldEndOffset();
 
-			tokenAppender.reset(frame, true);
-			tokenAppender.append(tokenTupleBuilder.getFieldEndOffsets(),
-					tokenTupleBuilder.getByteArray(), 0,
-					tokenTupleBuilder.getSize());
+            tokenAppender.reset(frame, true);
+            tokenAppender.append(tokenTupleBuilder.getFieldEndOffsets(), tokenTupleBuilder.getByteArray(), 0,
+                    tokenTupleBuilder.getSize());
 
-			searchKey.reset(tokenAccessor, 0);
+            searchKey.reset(tokenAccessor, 0);
 
-			invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx,
-					invListCursor);
+            invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursor);
 
-			invListCursor.pinPagesSync();
-			int checkIndex = 0;
-			while (invListCursor.hasNext()) {
-				invListCursor.next();
-				ITupleReference invListTuple = invListCursor.getTuple();
-				int invListElement = IntegerSerializerDeserializer.getInt(
-						invListTuple.getFieldData(0),
-						invListTuple.getFieldStart(0));
-				int checkInvListElement = checkListElements.get(i)
-						.get(checkIndex).intValue();
-				Assert.assertEquals(invListElement, checkInvListElement);
-				checkIndex++;
-			}
-			invListCursor.unpinPages();
-			Assert.assertEquals(checkIndex, checkListElements.get(i).size());
-		}
+            invListCursor.pinPagesSync();
+            int checkIndex = 0;
+            while (invListCursor.hasNext()) {
+                invListCursor.next();
+                ITupleReference invListTuple = invListCursor.getTuple();
+                int invListElement = IntegerSerializerDeserializer.getInt(invListTuple.getFieldData(0),
+                        invListTuple.getFieldStart(0));
+                int checkInvListElement = checkListElements.get(i).get(checkIndex).intValue();
+                Assert.assertEquals(invListElement, checkInvListElement);
+                checkIndex++;
+            }
+            invListCursor.unpinPages();
+            Assert.assertEquals(checkIndex, checkListElements.get(i).size());
+        }
 
-		// check that non-existing tokens have an empty inverted list
-		List<String> nonExistingTokens = new ArrayList<String>();
-		nonExistingTokens.add("watermelon");
-		nonExistingTokens.add("avocado");
-		nonExistingTokens.add("lemon");
+        // check that non-existing tokens have an empty inverted list
+        List<String> nonExistingTokens = new ArrayList<String>();
+        nonExistingTokens.add("watermelon");
+        nonExistingTokens.add("avocado");
+        nonExistingTokens.add("lemon");
 
-		for (int i = 0; i < nonExistingTokens.size(); i++) {
+        for (int i = 0; i < nonExistingTokens.size(); i++) {
 
-			tokenTupleBuilder.reset();
-			UTF8StringSerializerDeserializer.INSTANCE.serialize(
-					nonExistingTokens.get(i), tokenDos);
-			tokenTupleBuilder.addFieldEndOffset();
+            tokenTupleBuilder.reset();
+            UTF8StringSerializerDeserializer.INSTANCE.serialize(nonExistingTokens.get(i), tokenDos);
+            tokenTupleBuilder.addFieldEndOffset();
 
-			tokenAppender.reset(frame, true);
-			tokenAppender.append(tokenTupleBuilder.getFieldEndOffsets(),
-					tokenTupleBuilder.getByteArray(), 0,
-					tokenTupleBuilder.getSize());
+            tokenAppender.reset(frame, true);
+            tokenAppender.append(tokenTupleBuilder.getFieldEndOffsets(), tokenTupleBuilder.getByteArray(), 0,
+                    tokenTupleBuilder.getSize());
 
-			searchKey.reset(tokenAccessor, 0);
+            searchKey.reset(tokenAccessor, 0);
 
-			invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx,
-					invListCursor);
+            invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursor);
 
-			invListCursor.pinPagesSync();
-			Assert.assertEquals(invListCursor.hasNext(), false);
-			invListCursor.unpinPages();
-		}
+            invListCursor.pinPagesSync();
+            Assert.assertEquals(invListCursor.hasNext(), false);
+            invListCursor.unpinPages();
+        }
 
-		btree.close();
-		bufferCache.closeFile(btreeFileId);
-		bufferCache.closeFile(invListsFileId);
-		bufferCache.close();
-	}
+        btree.close();
+        bufferCache.closeFile(btreeFileId);
+        bufferCache.closeFile(invListsFileId);
+        bufferCache.close();
+    }
 
-	@AfterClass
-	public static void deinit() {
-		AbstractInvIndexTest.tearDown();
-	}
+    @AfterClass
+    public static void deinit() {
+        AbstractInvIndexTest.tearDown();
+    }
 }
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
index 161f20f..4ef8855 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
@@ -58,7 +58,7 @@
 		tokenFactory = new UTF8WordTokenFactory();
 		tokenizer = new DelimitedUTF8StringBinaryTokenizer(true, false,
 				tokenFactory);
-		searcher = new TOccurrenceSearcher(stageletCtx, invIndex, tokenizer);
+		searcher = new TOccurrenceSearcher(taskCtx, invIndex, tokenizer);
 		resultCursor = new SearchResultCursor(
 				searcher.createResultFrameTupleAccessor(),
 				searcher.createResultTupleReference());
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
index d9fef2c..c87ce9b 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
@@ -40,7 +40,7 @@
 		tokenFactory = new UTF8NGramTokenFactory();
 		tokenizer = new NGramUTF8StringBinaryTokenizer(3, false, true, false,
 				tokenFactory);
-		searcher = new TOccurrenceSearcher(stageletCtx, invIndex, tokenizer);
+		searcher = new TOccurrenceSearcher(taskCtx, invIndex, tokenizer);
 		resultCursor = new SearchResultCursor(
 				searcher.createResultFrameTupleAccessor(),
 				searcher.createResultTupleReference());
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
index 72c3f38..514b81f 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
@@ -23,7 +23,7 @@
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -67,7 +67,7 @@
     private static final int NUM_PAGES = 10;
     private static final int MAX_OPEN_FILES = 10;
     private static final int HYRACKS_FRAME_SIZE = 128;
-    private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+    private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
 
     // create an R-tree of two dimensions
     // fill the R-tree with random values using insertions
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
index 3060765..fb53b2a 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
@@ -27,7 +27,7 @@
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -71,7 +71,7 @@
     private static final int NUM_PAGES = 10;
     private static final int MAX_OPEN_FILES = 10;
     private static final int HYRACKS_FRAME_SIZE = 128;
-    private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+    private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
 
     // create an R-tree of two dimensions
     // fill the R-tree with random values using insertions
diff --git a/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java b/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java
index 680fd3b..80502eb 100644
--- a/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java
+++ b/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java
@@ -13,7 +13,7 @@
 import org.junit.Assert;
 import org.junit.Test;
 
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -24,25 +24,25 @@
 import edu.uci.ics.hyracks.test.support.TestUtils;
 
 public class BufferCacheTest {
-    protected static final List<String> openedFiles = new ArrayList<String>();    
+    protected static final List<String> openedFiles = new ArrayList<String>();
     protected static final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
     protected static final String tmpDir = System.getProperty("java.io.tmpdir");
-    protected static final String sep = System.getProperty("file.separator");    
-    
+    protected static final String sep = System.getProperty("file.separator");
+
     private static final int PAGE_SIZE = 256;
     private static final int NUM_PAGES = 10;
     private static final int MAX_OPEN_FILES = 20;
     private static final int HYRACKS_FRAME_SIZE = PAGE_SIZE;
-    private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
-    
+    private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+
     private static final Random rnd = new Random(50);
-    
+
     private String getFileName() {
         String fileName = tmpDir + sep + simpleDateFormat.format(new Date()) + openedFiles.size();
         openedFiles.add(fileName);
         return fileName;
-    }          
-    
+    }
+
     @Test
     public void simpleOpenPinCloseTest() throws HyracksDataException {
         TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
@@ -54,85 +54,84 @@
         int fileId = fmp.lookupFileId(file);
         int num = 10;
         int testPageId = 0;
-        
+
         bufferCache.openFile(fileId);
-        
+
         ICachedPage page = null;
-        
+
         // tryPin should fail
         page = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, testPageId));
         Assert.assertNull(page);
-        
+
         // pin page should succeed
         page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, testPageId), true);
         page.acquireWriteLatch();
-        try {            
-            for(int i = 0; i < num; i++) {
+        try {
+            for (int i = 0; i < num; i++) {
                 page.getBuffer().putInt(i * 4, i);
-            }               
-            
+            }
+
             // try pin should succeed         
             ICachedPage page2 = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, testPageId));
             Assert.assertNotNull(page2);
             bufferCache.unpin(page2);
-            
+
         } finally {
             page.releaseWriteLatch();
             bufferCache.unpin(page);
-        }                        
-        
+        }
+
         bufferCache.closeFile(fileId);
-        
+
         boolean exceptionThrown = false;
-        
+
         // tryPin should fail since file is not open
         try {
             page = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, testPageId));
-        } catch(HyracksDataException e) {
+        } catch (HyracksDataException e) {
             exceptionThrown = true;
         }
         Assert.assertTrue(exceptionThrown);
-        
+
         // pin should fail since file is not open
         exceptionThrown = false;
         try {
             page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, testPageId), false);
-        } catch(HyracksDataException e) {
+        } catch (HyracksDataException e) {
             exceptionThrown = true;
         }
         Assert.assertTrue(exceptionThrown);
-        
-        
+
         // open file again
         bufferCache.openFile(fileId);
-        
+
         // tryPin should succeed because page should still be cached        
-        page = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, testPageId));        
+        page = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, testPageId));
         Assert.assertNotNull(page);
         page.acquireReadLatch();
         try {
             // verify contents of page
-            for(int i = 0; i < num; i++) {
+            for (int i = 0; i < num; i++) {
                 Assert.assertEquals(page.getBuffer().getInt(i * 4), i);
             }
         } finally {
             page.releaseReadLatch();
             bufferCache.unpin(page);
-        }                
-        
-        bufferCache.closeFile(fileId);        
+        }
+
+        bufferCache.closeFile(fileId);
         bufferCache.close();
     }
-        
+
     @Test
     public void simpleMaxOpenFilesTest() throws HyracksDataException {
         TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
         IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
         IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
-        
+
         List<Integer> fileIds = new ArrayList<Integer>();
-        
-        for(int i = 0; i < MAX_OPEN_FILES; i++) {
+
+        for (int i = 0; i < MAX_OPEN_FILES; i++) {
             String fileName = getFileName();
             FileReference file = new FileReference(new File(fileName));
             bufferCache.createFile(file);
@@ -140,9 +139,9 @@
             bufferCache.openFile(fileId);
             fileIds.add(fileId);
         }
-        
+
         boolean exceptionThrown = false;
-        
+
         // since all files are open, next open should fail
         try {
             String fileName = getFileName();
@@ -150,16 +149,16 @@
             bufferCache.createFile(file);
             int fileId = fmp.lookupFileId(file);
             bufferCache.openFile(fileId);
-        } catch(HyracksDataException e) {
+        } catch (HyracksDataException e) {
             exceptionThrown = true;
         }
         Assert.assertTrue(exceptionThrown);
-        
+
         // close a random file
         int ix = Math.abs(rnd.nextInt()) % fileIds.size();
         bufferCache.closeFile(fileIds.get(ix));
         fileIds.remove(ix);
-        
+
         // now open should succeed again
         exceptionThrown = false;
         try {
@@ -169,58 +168,58 @@
             int fileId = fmp.lookupFileId(file);
             bufferCache.openFile(fileId);
             fileIds.add(fileId);
-            
-        } catch(HyracksDataException e) {
+
+        } catch (HyracksDataException e) {
             exceptionThrown = true;
         }
         Assert.assertFalse(exceptionThrown);
-        
-        for(Integer i : fileIds) {
+
+        for (Integer i : fileIds) {
             bufferCache.closeFile(i.intValue());
         }
-        
+
         bufferCache.close();
     }
-    
+
     @Test
     public void contentCheckingMaxOpenFilesTest() throws HyracksDataException {
         TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
         IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
         IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
-        
+
         List<Integer> fileIds = new ArrayList<Integer>();
-        Map<Integer, ArrayList<Integer>> pageContents = new HashMap<Integer, ArrayList<Integer>>();        
+        Map<Integer, ArrayList<Integer>> pageContents = new HashMap<Integer, ArrayList<Integer>>();
         int num = 10;
-        int testPageId = 0;        
-        
+        int testPageId = 0;
+
         // open max number of files and write some stuff into their first page
-        for(int i = 0; i < MAX_OPEN_FILES; i++) {
+        for (int i = 0; i < MAX_OPEN_FILES; i++) {
             String fileName = getFileName();
             FileReference file = new FileReference(new File(fileName));
             bufferCache.createFile(file);
             int fileId = fmp.lookupFileId(file);
             bufferCache.openFile(fileId);
             fileIds.add(fileId);
-            
-            ICachedPage page = null;            
+
+            ICachedPage page = null;
             page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, testPageId), true);
             page.acquireWriteLatch();
-            try {            
-                ArrayList<Integer> values = new ArrayList<Integer>();                
-                for(int j = 0; j < num; j++) {
+            try {
+                ArrayList<Integer> values = new ArrayList<Integer>();
+                for (int j = 0; j < num; j++) {
                     int x = Math.abs(rnd.nextInt());
                     page.getBuffer().putInt(j * 4, x);
                     values.add(x);
                 }
-                pageContents.put(fileId, values);                
+                pageContents.put(fileId, values);
             } finally {
                 page.releaseWriteLatch();
                 bufferCache.unpin(page);
-            }            
-        }        
-        
+            }
+        }
+
         boolean exceptionThrown = false;
-        
+
         // since all files are open, next open should fail
         try {
             String fileName = getFileName();
@@ -228,82 +227,82 @@
             bufferCache.createFile(file);
             int fileId = fmp.lookupFileId(file);
             bufferCache.openFile(fileId);
-        } catch(HyracksDataException e) {
+        } catch (HyracksDataException e) {
             exceptionThrown = true;
         }
-        Assert.assertTrue(exceptionThrown);        
-        
+        Assert.assertTrue(exceptionThrown);
+
         // close a few random files
         ArrayList<Integer> closedFileIds = new ArrayList<Integer>();
         int filesToClose = 5;
-        for(int i = 0; i < filesToClose; i++) {
-            int ix = Math.abs(rnd.nextInt()) % fileIds.size();            
+        for (int i = 0; i < filesToClose; i++) {
+            int ix = Math.abs(rnd.nextInt()) % fileIds.size();
             bufferCache.closeFile(fileIds.get(ix));
             closedFileIds.add(fileIds.get(ix));
             fileIds.remove(ix);
-        }        
-        
+        }
+
         // now open a few new files
-        for(int i = 0; i < filesToClose; i++) {
+        for (int i = 0; i < filesToClose; i++) {
             String fileName = getFileName();
             FileReference file = new FileReference(new File(fileName));
             bufferCache.createFile(file);
             int fileId = fmp.lookupFileId(file);
             bufferCache.openFile(fileId);
             fileIds.add(fileId);
-        }        
-        
+        }
+
         // since all files are open, next open should fail
         try {
             String fileName = getFileName();
             FileReference file = new FileReference(new File(fileName));
             bufferCache.createFile(file);
             int fileId = fmp.lookupFileId(file);
-            bufferCache.openFile(fileId);            
-        } catch(HyracksDataException e) {
+            bufferCache.openFile(fileId);
+        } catch (HyracksDataException e) {
             exceptionThrown = true;
         }
-        Assert.assertTrue(exceptionThrown);       
-        
+        Assert.assertTrue(exceptionThrown);
+
         // close a few random files again        
-        for(int i = 0; i < filesToClose; i++) {
-            int ix = Math.abs(rnd.nextInt()) % fileIds.size();            
+        for (int i = 0; i < filesToClose; i++) {
+            int ix = Math.abs(rnd.nextInt()) % fileIds.size();
             bufferCache.closeFile(fileIds.get(ix));
             closedFileIds.add(fileIds.get(ix));
             fileIds.remove(ix);
         }
-        
+
         // now open those closed files again and verify their contents
-        for(int i = 0; i < filesToClose; i++) {
+        for (int i = 0; i < filesToClose; i++) {
             int closedFileId = closedFileIds.get(i);
             bufferCache.openFile(closedFileId);
             fileIds.add(closedFileId);
-            
+
             // pin first page and verify contents
-            ICachedPage page = null;            
+            ICachedPage page = null;
             page = bufferCache.pin(BufferedFileHandle.getDiskPageId(closedFileId, testPageId), false);
             page.acquireReadLatch();
-            try {            
-                ArrayList<Integer> values = pageContents.get(closedFileId);                
-                for(int j = 0; j < values.size(); j++) {
-                    Assert.assertEquals(values.get(j).intValue(), page.getBuffer().getInt(j * 4));                    
-                }      
+            try {
+                ArrayList<Integer> values = pageContents.get(closedFileId);
+                for (int j = 0; j < values.size(); j++) {
+                    Assert.assertEquals(values.get(j).intValue(), page.getBuffer().getInt(j * 4));
+                }
             } finally {
                 page.releaseReadLatch();
                 bufferCache.unpin(page);
-            }            
-        }         
-        
-        for(Integer i : fileIds) {
+            }
+        }
+
+        for (Integer i : fileIds) {
             bufferCache.closeFile(i.intValue());
         }
-        
+
         bufferCache.close();
     }
-    
+
     @AfterClass
     public static void cleanup() throws Exception {
-        for(String s : openedFiles) {
+        for (String s : openedFiles) {
             File f = new File(s);
             f.deleteOnExit();
         }