Merged in hyracks_scheduling branch

git-svn-id: https://hyracks.googlecode.com/svn/trunk/hyracks@319 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/AbsoluteLocationConstraint.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/AbsoluteLocationConstraint.java
deleted file mode 100644
index bbd4b4f..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/AbsoluteLocationConstraint.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.constraints;
-
-public class AbsoluteLocationConstraint extends LocationConstraint {
-    private static final long serialVersionUID = 1L;
-
-    private String nodeId;
-
-    public AbsoluteLocationConstraint() {
-    }
-
-    public AbsoluteLocationConstraint(String nodeId) {
-        this.nodeId = nodeId;
-    }
-
-    @Override
-    public LocationConstraintType getConstraintType() {
-        return LocationConstraintType.ABSOLUTE;
-    }
-
-    public String getLocationId() {
-        return nodeId;
-    }
-
-    public void setLocationId(String nodeId) {
-        this.nodeId = nodeId;
-    }
-
-    public String toString() {
-        return getConstraintType() + ":" + nodeId;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/ChoiceLocationConstraint.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/ChoiceLocationConstraint.java
deleted file mode 100644
index e07341b..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/ChoiceLocationConstraint.java
+++ /dev/null
@@ -1,41 +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 java.util.Arrays;
-
-public class ChoiceLocationConstraint extends LocationConstraint {
-    private static final long serialVersionUID = 1L;
-
-    private LocationConstraint[] choices;
-
-    public ChoiceLocationConstraint(LocationConstraint... choices) {
-        this.choices = choices;
-    }
-
-    public LocationConstraint[] getChoices() {
-        return choices;
-    }
-
-    @Override
-    public String toString() {
-        return "choice(" + Arrays.deepToString(choices) + ")";
-    }
-
-    @Override
-    public LocationConstraintType getConstraintType() {
-        return LocationConstraintType.CHOICE;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/ExplicitPartitionConstraint.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/ExplicitPartitionConstraint.java
deleted file mode 100644
index 16b7b24..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/ExplicitPartitionConstraint.java
+++ /dev/null
@@ -1,41 +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 java.util.Arrays;
-
-public class ExplicitPartitionConstraint extends PartitionConstraint {
-    private static final long serialVersionUID = 1L;
-
-    private final LocationConstraint[] locationConstraints;
-
-    public ExplicitPartitionConstraint(LocationConstraint[] locationConstraints) {
-        this.locationConstraints = locationConstraints;
-    }
-
-    public LocationConstraint[] getLocationConstraints() {
-        return locationConstraints;
-    }
-
-    @Override
-    public String toString() {
-        return Arrays.deepToString(locationConstraints);
-    }
-
-    @Override
-    public PartitionConstraintType getPartitionConstraintType() {
-        return PartitionConstraintType.EXPLICIT;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintExpressionAcceptor.java
similarity index 70%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintExpressionAcceptor.java
index 75edf2b..7e74814 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintExpressionAcceptor.java
@@ -12,10 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.web.handlers.util;
+package edu.uci.ics.hyracks.api.constraints;
 
-import org.json.JSONObject;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
 
-public interface IJSONOutputFunction {
-    public JSONObject invoke(String[] arguments) throws Exception;
+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/LocationConstraint.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/LocationConstraint.java
deleted file mode 100644
index 1228b90..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/LocationConstraint.java
+++ /dev/null
@@ -1,28 +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 java.io.Serializable;
-
-public abstract class LocationConstraint implements Serializable {
-    private static final long serialVersionUID = 1L;
-
-    public enum LocationConstraintType {
-        ABSOLUTE,
-        CHOICE
-    }
-
-    public abstract LocationConstraintType getConstraintType();
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraint.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraint.java
deleted file mode 100644
index 82a6f7b..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraint.java
+++ /dev/null
@@ -1,28 +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 java.io.Serializable;
-
-public abstract class PartitionConstraint implements Serializable {
-    private static final long serialVersionUID = 1L;
-
-    public abstract PartitionConstraintType getPartitionConstraintType();
-
-    public enum PartitionConstraintType {
-        EXPLICIT,
-        COUNT
-    }
-}
\ 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
new file mode 100644
index 0000000..309fca1
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraintHelper.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.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));
+    }
+
+    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)));
+        }
+    }
+
+    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)));
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionCountConstraint.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionCountConstraint.java
deleted file mode 100644
index f4fdf67..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionCountConstraint.java
+++ /dev/null
@@ -1,39 +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;
-
-public class PartitionCountConstraint extends PartitionConstraint {
-    private static final long serialVersionUID = 1L;
-
-    private final int count;
-
-    public PartitionCountConstraint(int count) {
-        this.count = count;
-    }
-
-    public int getCount() {
-        return count;
-    }
-
-    @Override
-    public PartitionConstraintType getPartitionConstraintType() {
-        return PartitionConstraintType.COUNT;
-    }
-
-    @Override
-    public String toString() {
-        return "count(" + count + ")";
-    }
-}
\ 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
new file mode 100644
index 0000000..c307e3e
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/BelongsToExpression.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.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/ConstantExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ConstantExpression.java
new file mode 100644
index 0000000..6e10911
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ConstantExpression.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.constraints.expressions;
+
+import java.util.Collection;
+
+public class ConstantExpression extends ConstraintExpression {
+    private static final long serialVersionUID = 1L;
+
+    private final Object value;
+
+    public ConstantExpression(Object value) {
+        this.value = value;
+    }
+
+    @Override
+    public ExpressionTag getTag() {
+        return ExpressionTag.CONSTANT;
+    }
+
+    public Object getValue() {
+        return value;
+    }
+
+    @Override
+    public void getChildren(Collection<ConstraintExpression> children) {
+    }
+
+    @Override
+    protected void toString(StringBuilder buffer) {
+        buffer.append(getTag()).append('[').append(value).append(':').append(value.getClass().getName()).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
new file mode 100644
index 0000000..afd5051
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ConstraintExpression.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.constraints.expressions;
+
+import java.io.Serializable;
+import java.util.Collection;
+
+public abstract class ConstraintExpression implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public enum ExpressionTag {
+        CONSTANT,
+        PARTITION_COUNT,
+        RESOURCE_SUM,
+        PARTITION_LOCATION,
+        ENUMERATED_SET,
+        BELONGS_TO,
+        RELATIONAL,
+    }
+
+    public abstract ExpressionTag getTag();
+
+    public abstract void getChildren(Collection<ConstraintExpression> children);
+
+    @Override
+    public final String toString() {
+        StringBuilder buffer = new StringBuilder();
+        toString(buffer);
+        return buffer.toString();
+    }
+
+    protected abstract void toString(StringBuilder buffer);
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..d0ce706
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/EnumeratedCollectionExpression.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.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/constraints/expressions/PartitionCountExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionCountExpression.java
new file mode 100644
index 0000000..c538ff7
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionCountExpression.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.constraints.expressions;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+
+public final class PartitionCountExpression extends ConstraintExpression {
+    private static final long serialVersionUID = 1L;
+
+    private final OperatorDescriptorId opId;
+
+    public PartitionCountExpression(OperatorDescriptorId opId) {
+        this.opId = opId;
+    }
+
+    @Override
+    public ExpressionTag getTag() {
+        return ExpressionTag.PARTITION_COUNT;
+    }
+
+    public OperatorDescriptorId getOperatorDescriptorId() {
+        return opId;
+    }
+
+    @Override
+    public void getChildren(Collection<ConstraintExpression> children) {
+    }
+
+    @Override
+    protected void toString(StringBuilder buffer) {
+        buffer.append(getTag()).append('(').append(opId.toString()).append(')');
+    }
+}
\ 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
new file mode 100644
index 0000000..50e4e47
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionLocationExpression.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.constraints.expressions;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+
+public final class PartitionLocationExpression extends ConstraintExpression {
+    private static final long serialVersionUID = 1L;
+
+    private final OperatorDescriptorId opId;
+    private final int partition;
+
+    public PartitionLocationExpression(OperatorDescriptorId opId, int partition) {
+        this.opId = opId;
+        this.partition = partition;
+    }
+
+    @Override
+    public ExpressionTag getTag() {
+        return ExpressionTag.PARTITION_LOCATION;
+    }
+
+    public OperatorDescriptorId getOperatorDescriptorId() {
+        return opId;
+    }
+
+    public int getPartition() {
+        return partition;
+    }
+
+    @Override
+    public void getChildren(Collection<ConstraintExpression> children) {
+    }
+
+    @Override
+    protected void toString(StringBuilder buffer) {
+        buffer.append(getTag()).append('(').append(opId.toString()).append(", ").append(partition).append(')');
+    }
+}
\ 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
new file mode 100644
index 0000000..d90bce9
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/RelationalExpression.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.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
new file mode 100644
index 0000000..3fbdd08
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ResourceSumExpression.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.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/control/CCConfig.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/CCConfig.java
index 7cdaee1..4d9b5a0 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/CCConfig.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/CCConfig.java
@@ -32,6 +32,6 @@
     @Option(name = "-profile-dump-period", usage = "Sets the time duration between two profile dumps from each node controller in milliseconds. 0 to disable. (default: 0)")
     public int profileDumpPeriod = 0;
 
-    @Option(name = "-use-jol", usage = "Forces Hyracks to use the JOL based scheduler (default: false)")
-    public boolean useJOL = false;
+    @Option(name = "-default-max-job-attempts", usage = "Sets the default number of job attempts allowed if not specified in the job specification. (default: 5)")
+    public int defaultMaxJobAttempts = 5;
 }
\ 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
index f577db2..9023cee 100644
--- 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
@@ -15,20 +15,23 @@
 package edu.uci.ics.hyracks.api.control;
 
 import java.rmi.Remote;
-import java.util.Map;
+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,
-            Map<String, Long> statistics) 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, Map<UUID, Map<String, Long>> counterDump) 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
index 1842959..d0269fc 100644
--- 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
@@ -32,16 +32,16 @@
     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, Set<Integer>> opPartitions)
+            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, Set<Integer>> opPartitions,
+            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, UUID stageId) throws Exception;
+    public void abortJoblet(UUID jobId, int attempt) throws Exception;
 
     public void cleanUpJob(UUID jobId) throws Exception;
 
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 63bed19..362306a 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
@@ -22,9 +22,11 @@
 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.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobPlan;
 
 /**
  * Connector that connects operators in a Job.
@@ -84,6 +86,16 @@
             throws HyracksDataException;
 
     /**
+     * Contribute any scheduling constraints imposed by this connector
+     * 
+     * @param constraintAcceptor
+     *            - Constraint Acceptor
+     * @param plan
+     *            - Job Plan
+     */
+    public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan);
+
+    /**
      * 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 ad51370..e58ac99 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
@@ -19,8 +19,9 @@
 import org.json.JSONException;
 import org.json.JSONObject;
 
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobPlan;
 
 /**
  * Descriptor for operators in Hyracks.
@@ -50,21 +51,6 @@
     public int getOutputArity();
 
     /**
-     * Returns the partition constraint requested for scheduling this operator. This value is set as part of Job creation by the client.
-     * 
-     * @return the partition constraint.
-     */
-    public PartitionConstraint getPartitionConstraint();
-
-    /**
-     * Sets the partition constraint.
-     * 
-     * @param partitionConstraint
-     *            - partition constraint.
-     */
-    public void setPartitionConstraint(PartitionConstraint partitionConstraint);
-
-    /**
      * Gets the output record descriptor
      * 
      * @return Array of RecordDescriptor, one per output.
@@ -80,6 +66,16 @@
     public void contributeTaskGraph(IActivityGraphBuilder builder);
 
     /**
+     * Contributes any scheduling constraints imposed by this operator.
+     * 
+     * @param constraintAcceptor
+     *            - Constraint Acceptor
+     * @param plan
+     *            - Job Plan
+     */
+    public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan);
+
+    /**
      * Translates this operator descriptor to JSON.
      */
     public JSONObject toJSON() throws JSONException;
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
index 5d8ef1d..13da402 100644
--- 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
@@ -21,18 +21,21 @@
 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;
-import edu.uci.ics.hyracks.api.job.JobFlag;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 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;
@@ -53,9 +56,9 @@
 
     private final Map<OperatorDescriptorId, List<ActivityNodeId>> operatorOutputMap;
 
-    private JobStage endStage;
-
-    public JobPlan(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
+    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>();
@@ -68,6 +71,14 @@
         operatorOutputMap = new HashMap<OperatorDescriptorId, List<ActivityNodeId>>();
     }
 
+    public String getApplicationName() {
+        return appName;
+    }
+
+    public UUID getJobId() {
+        return jobId;
+    }
+
     public JobSpecification getJobSpecification() {
         return jobSpec;
     }
@@ -108,14 +119,6 @@
         return operatorOutputMap;
     }
 
-    public void setEndStage(JobStage endStage) {
-        this.endStage = endStage;
-    }
-
-    public JobStage getEndStage() {
-        return endStage;
-    }
-
     public List<IConnectorDescriptor> getTaskInputs(ActivityNodeId hanId) {
         List<Integer> inputIndexes = taskInputMap.get(hanId);
         if (inputIndexes == null) {
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 fce9c22..59072f8 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
@@ -17,13 +17,16 @@
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 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.constraints.expressions.ConstraintExpression;
 import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
@@ -48,6 +51,10 @@
 
     private final Map<String, Serializable> properties;
 
+    private final Set<ConstraintExpression> userConstraints;
+
+    private int maxAttempts;
+
     public JobSpecification() {
         roots = new ArrayList<OperatorDescriptorId>();
         opMap = new HashMap<OperatorDescriptorId, IOperatorDescriptor>();
@@ -56,6 +63,7 @@
         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>();
     }
 
     public void addRoot(IOperatorDescriptor op) {
@@ -71,11 +79,11 @@
                         new Pair<IOperatorDescriptor, Integer>(producerOp, producerPort),
                         new Pair<IOperatorDescriptor, Integer>(consumerOp, consumerPort)));
     }
-    
+
     public void setProperty(String name, Serializable value) {
         properties.put(name, value);
     }
-    
+
     public Serializable getProperty(String name) {
         return properties.get(name);
     }
@@ -165,6 +173,22 @@
         return roots;
     }
 
+    public void setMaxAttempts(int maxAttempts) {
+        this.maxAttempts = maxAttempts;
+    }
+
+    public int getMaxAttempts() {
+        return maxAttempts;
+    }
+
+    public void addUserConstraint(ConstraintExpression constraint) {
+        userConstraints.add(constraint);
+    }
+
+    public Set<ConstraintExpression> getUserConstraints() {
+        return userConstraints;
+    }
+
     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) {
@@ -180,8 +204,6 @@
 
         for (Map.Entry<OperatorDescriptorId, IOperatorDescriptor> e : opMap.entrySet()) {
             buffer.append(e.getKey().getId()).append(" : ").append(e.getValue().toString()).append("\n");
-            IOperatorDescriptor op = e.getValue();
-            buffer.append("   Partition Constraint: ").append(op.getPartitionConstraint()).append("\n");
             List<IConnectorDescriptor> inputs = opInputMap.get(e.getKey());
             if (inputs != null && !inputs.isEmpty()) {
                 buffer.append("   Inputs:\n");
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobStatus.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobStatus.java
index 977f41a..aa6fb68 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobStatus.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobStatus.java
@@ -18,4 +18,5 @@
     INITIALIZED,
     RUNNING,
     TERMINATED,
+    FAILURE,
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/counters/ICounterContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/counters/ICounterContext.java
index 2992757..4be02d7 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/counters/ICounterContext.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/counters/ICounterContext.java
@@ -21,13 +21,6 @@
  */
 public interface ICounterContext {
     /**
-     * Returns the fully-qualified context name
-     * 
-     * @return fully-qualified context name.
-     */
-    public String getContextName();
-
-    /**
      * Get a counter with the specified name.
      * 
      * @param name
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/AbstractProfile.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/AbstractProfile.java
new file mode 100644
index 0000000..6ef57a6
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/AbstractProfile.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.job.profiling.om;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+public abstract class AbstractProfile implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    protected final Map<String, Long> counters;
+
+    public AbstractProfile() {
+        counters = new HashMap<String, Long>();
+    }
+
+    public Map<String, Long> getCounters() {
+        return counters;
+    }
+
+    public abstract JSONObject toJSON() throws JSONException;
+
+    protected void populateCounters(JSONObject jo) throws JSONException {
+        for (Map.Entry<String, Long> e : counters.entrySet()) {
+            JSONObject jpe = new JSONObject();
+            jpe.put("name", e.getKey());
+            jpe.put("value", e.getValue());
+            jo.accumulate("counters", jpe);
+        }
+    }
+
+    protected void merge(AbstractProfile profile) {
+        counters.putAll(profile.counters);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobProfile.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobProfile.java
new file mode 100644
index 0000000..53ad0ad
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobProfile.java
@@ -0,0 +1,62 @@
+package edu.uci.ics.hyracks.api.job.profiling.om;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+public class JobProfile extends AbstractProfile {
+    private static final long serialVersionUID = 1L;
+
+    private final UUID jobId;
+
+    private final int attempt;
+
+    private final Map<String, JobletProfile> jobletProfiles;
+
+    public JobProfile(UUID jobId, int attempt) {
+        this.jobId = jobId;
+        this.attempt = attempt;
+        jobletProfiles = new HashMap<String, JobletProfile>();
+    }
+
+    public UUID getJobId() {
+        return jobId;
+    }
+
+    public int getAttempt() {
+        return attempt;
+    }
+
+    public Map<String, JobletProfile> getJobletProfiles() {
+        return jobletProfiles;
+    }
+
+    @Override
+    public JSONObject toJSON() throws JSONException {
+        JSONObject json = new JSONObject();
+
+        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());
+        }
+
+        return json;
+    }
+
+    public void merge(JobProfile other) {
+        super.merge(this);
+        for (JobletProfile jp : other.jobletProfiles.values()) {
+            if (jobletProfiles.containsKey(jp.getNodeId())) {
+                jobletProfiles.get(jp.getNodeId()).merge(jp);
+            } else {
+                jobletProfiles.put(jp.getNodeId(), jp);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobletProfile.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobletProfile.java
new file mode 100644
index 0000000..66a203c
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobletProfile.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.api.job.profiling.om;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+public class JobletProfile extends AbstractProfile {
+    private static final long serialVersionUID = 1L;
+
+    private final String nodeId;
+
+    private final Map<UUID, StageletProfile> stageletProfiles;
+
+    public JobletProfile(String nodeId) {
+        this.nodeId = nodeId;
+        stageletProfiles = new HashMap<UUID, StageletProfile>();
+    }
+
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public Map<UUID, StageletProfile> getStageletProfiles() {
+        return stageletProfiles;
+    }
+
+    @Override
+    public JSONObject toJSON() throws JSONException {
+        JSONObject json = new JSONObject();
+
+        json.put("type", "joblet-profile");
+        json.put("node-id", nodeId.toString());
+        populateCounters(json);
+        for (StageletProfile p : stageletProfiles.values()) {
+            json.accumulate("stagelets", p.toJSON());
+        }
+
+        return json;
+    }
+
+    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);
+            } else {
+                stageletProfiles.put(sp.getStageId(), sp);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/StageProgress.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/StageletProfile.java
similarity index 60%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/StageProgress.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/StageletProfile.java
index 102a011..cb43f66 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/StageProgress.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/StageletProfile.java
@@ -12,35 +12,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc;
+package edu.uci.ics.hyracks.api.job.profiling.om;
 
-import java.util.HashSet;
-import java.util.Set;
 import java.util.UUID;
 
-public class StageProgress {
+import org.json.JSONException;
+import org.json.JSONObject;
+
+public class StageletProfile extends AbstractProfile {
+    private static final long serialVersionUID = 1L;
+
     private final UUID stageId;
 
-    private final Set<String> pendingNodes;
-
-    public StageProgress(UUID stageId) {
+    public StageletProfile(UUID stageId) {
         this.stageId = stageId;
-        pendingNodes = new HashSet<String>();
     }
 
     public UUID getStageId() {
         return stageId;
     }
 
-    public void addPendingNodes(Set<String> nodes) {
-        pendingNodes.addAll(nodes);
-    }
+    @Override
+    public JSONObject toJSON() throws JSONException {
+        JSONObject json = new JSONObject();
 
-    public void markNodeComplete(String nodeId) {
-        pendingNodes.remove(nodeId);
-    }
+        json.put("type", "joblet-profile");
+        json.put("stage-id", stageId.toString());
+        populateCounters(json);
 
-    public boolean stageComplete() {
-        return pendingNodes.isEmpty();
+        return json;
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/pom.xml b/hyracks-control-cc/pom.xml
index e639a1f..9af88b1 100644
--- a/hyracks-control-cc/pom.xml
+++ b/hyracks-control-cc/pom.xml
@@ -32,13 +32,6 @@
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
-  		<groupId>jol</groupId>
-  		<artifactId>jol</artifactId>
-  		<version>1.0.0</version>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>
-  	<dependency>
   		<groupId>org.eclipse.jetty</groupId>
   		<artifactId>jetty-server</artifactId>
   		<version>8.0.0.M1</version>
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 f577c97..a4aa482 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
@@ -14,72 +14,56 @@
  */
 package edu.uci.ics.hyracks.control.cc;
 
-import java.io.ByteArrayOutputStream;
 import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.ObjectOutputStream;
-import java.io.OutputStream;
-import java.io.PrintWriter;
 import java.rmi.registry.LocateRegistry;
 import java.rmi.registry.Registry;
-import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Hashtable;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
 import java.util.UUID;
-import java.util.Vector;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Executors;
-import java.util.concurrent.Semaphore;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-
-import jol.core.Runtime;
-import jol.core.Runtime.DebugLevel;
-
-import org.apache.commons.io.IOUtils;
-import org.eclipse.jetty.http.HttpMethods;
-import org.eclipse.jetty.server.Handler;
-import org.eclipse.jetty.server.Request;
-import org.eclipse.jetty.server.handler.AbstractHandler;
-import org.eclipse.jetty.server.handler.ContextHandler;
-import org.json.JSONObject;
-
 import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
 import edu.uci.ics.hyracks.api.client.IHyracksClientInterface;
-import edu.uci.ics.hyracks.api.comm.Endpoint;
 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.NodeParameters;
-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.JobFlag;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
+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;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.ApplicationDestroyEvent;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.ApplicationStartEvent;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobStatusConditionVariableEvent;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobStatusEvent;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.JobCreateEvent;
+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.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.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.cc.web.handlers.util.IJSONOutputFunction;
-import edu.uci.ics.hyracks.control.cc.web.handlers.util.JSONOutputRequestHandler;
-import edu.uci.ics.hyracks.control.cc.web.handlers.util.RoutingHandler;
 import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
-import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
 
 public class ClusterControllerService extends AbstractRemoteService implements IClusterController,
@@ -96,32 +80,31 @@
 
     private final ServerContext serverCtx;
 
-    private WebServer webServer;
+    private final WebServer webServer;
 
     private ClusterControllerInfo info;
 
-    private final IJobManager jobManager;
+    private final Map<UUID, JobRun> runMap;
+
+    private final JobQueue jobQueue;
+
+    private final IScheduler scheduler;
 
     private final Executor taskExecutor;
 
     private final Timer timer;
 
-    private Runtime jolRuntime;
-
     public ClusterControllerService(CCConfig ccConfig) throws Exception {
         this.ccConfig = ccConfig;
         nodeRegistry = new LinkedHashMap<String, NodeControllerState>();
         applications = new Hashtable<String, CCApplicationContext>();
         serverCtx = new ServerContext(ServerContext.ServerType.CLUSTER_CONTROLLER, new File(
                 ClusterControllerService.class.getName()));
-        Set<DebugLevel> jolDebugLevel = LOGGER.isLoggable(Level.FINE) ? Runtime.DEBUG_ALL : new HashSet<DebugLevel>();
-        jolRuntime = (Runtime) Runtime.create(jolDebugLevel, System.err);
-        jobManager = new JOLJobManagerImpl(this, jolRuntime);
         taskExecutor = Executors.newCachedThreadPool();
-        webServer = new WebServer();
-        webServer.addHandler(getAdminConsoleHandler());
-        webServer.addHandler(getApplicationInstallationHandler());
-        webServer.addHandler(getRestAPIHandler());
+        webServer = new WebServer(this);
+        runMap = new HashMap<UUID, JobRun>();
+        jobQueue = new JobQueue();
+        scheduler = new NaiveScheduler(this);
         this.timer = new Timer(true);
     }
 
@@ -146,35 +129,49 @@
         LOGGER.log(Level.INFO, "Stopped ClusterControllerService");
     }
 
-    @Override
-    public UUID createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
-        CCApplicationContext appCtx = getApplicationContext(appName);
-        if (appCtx == null) {
-            throw new HyracksException("No application with id " + appName + " found");
-        }
-        UUID jobId = UUID.randomUUID();
-        JobSpecification specification = appCtx.createJobSpecification(jobId, jobSpec);
-        jobManager.createJob(jobId, appName, specification, jobFlags);
-        appCtx.notifyJobCreation(jobId, specification);
-        return jobId;
+    public Map<String, CCApplicationContext> getApplicationMap() {
+        return applications;
     }
 
-    public synchronized CCApplicationContext getApplicationContext(String appName) {
-        return applications.get(appName);
+    public Map<UUID, JobRun> getRunMap() {
+        return runMap;
+    }
+
+    public JobQueue getJobQueue() {
+        return jobQueue;
+    }
+
+    public IScheduler getScheduler() {
+        return scheduler;
+    }
+
+    public Executor getExecutor() {
+        return taskExecutor;
+    }
+
+    public Map<String, NodeControllerState> getNodeMap() {
+        return nodeRegistry;
+    }
+
+    public CCConfig getConfig() {
+        return ccConfig;
+    }
+
+    @Override
+    public UUID createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+        UUID jobId = UUID.randomUUID();
+        JobCreateEvent jce = new JobCreateEvent(this, jobId, appName, jobSpec, jobFlags);
+        jobQueue.schedule(jce);
+        jce.sync();
+        return jobId;
     }
 
     @Override
     public NodeParameters registerNode(INodeController nodeController) throws Exception {
         String id = nodeController.getId();
         NodeControllerState state = new NodeControllerState(nodeController);
-        synchronized (this) {
-            if (nodeRegistry.containsKey(id)) {
-                throw new Exception("Node with this name already registered.");
-            }
-            nodeRegistry.put(id, state);
-        }
+        jobQueue.scheduleAndSync(new RegisterNodeEvent(this, id, state));
         nodeController.notifyRegistration(this);
-        jobManager.registerNode(id);
         LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
         NodeParameters params = new NodeParameters();
         params.setClusterControllerInfo(info);
@@ -186,188 +183,49 @@
     @Override
     public void unregisterNode(INodeController nodeController) throws Exception {
         String id = nodeController.getId();
-        synchronized (this) {
-            nodeRegistry.remove(id);
-        }
+        jobQueue.scheduleAndSync(new UnregisterNodeEvent(this, id));
         LOGGER.log(Level.INFO, "Unregistered INodeController");
     }
 
-    public synchronized NodeControllerState lookupNode(String id) throws Exception {
-        return nodeRegistry.get(id);
-    }
-
-    public Executor getExecutor() {
-        return taskExecutor;
-    }
-
-    public synchronized void notifyJobComplete(final UUID jobId) {
-        for (final NodeControllerState ns : nodeRegistry.values()) {
-            taskExecutor.execute(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        ns.getNodeController().cleanUpJob(jobId);
-                    } catch (Exception e) {
-                    }
-                }
-            });
-        }
-    }
-
     @Override
-    public void notifyStageletComplete(UUID jobId, UUID stageId, int attempt, String nodeId,
-            Map<String, Long> statistics) throws Exception {
-        jobManager.notifyStageletComplete(jobId, stageId, attempt, nodeId, statistics);
+    public void notifyStageletComplete(UUID jobId, UUID stageId, int attempt, String nodeId, StageletProfile statistics)
+            throws Exception {
+        StageletCompleteEvent sce = new StageletCompleteEvent(this, jobId, stageId, attempt, nodeId);
+        jobQueue.schedule(sce);
     }
 
     @Override
     public void notifyStageletFailure(UUID jobId, UUID stageId, int attempt, String nodeId) throws Exception {
-        jobManager.notifyStageletFailure(jobId, stageId, attempt, nodeId);
+        StageletFailureEvent sfe = new StageletFailureEvent(this, jobId, stageId, attempt, nodeId);
+        jobQueue.schedule(sfe);
     }
 
     @Override
     public JobStatus getJobStatus(UUID jobId) throws Exception {
-        return jobManager.getJobStatus(jobId);
+        GetJobStatusEvent gse = new GetJobStatusEvent(this, jobId);
+        jobQueue.scheduleAndSync(gse);
+        return gse.getStatus();
     }
 
     @Override
     public void start(UUID jobId) throws Exception {
-        jobManager.start(jobId);
+        JobStartEvent jse = new JobStartEvent(this, jobId);
+        jobQueue.scheduleAndSync(jse);
     }
 
     @Override
     public void waitForCompletion(UUID jobId) throws Exception {
-        jobManager.waitForCompletion(jobId);
+        GetJobStatusConditionVariableEvent e = new GetJobStatusConditionVariableEvent(this, jobId);
+        jobQueue.scheduleAndSync(e);
+        IJobStatusConditionVariable var = e.getConditionVariable();
+        if (var != null) {
+            var.waitForCompletion();
+        }
     }
 
     @Override
-    public void reportProfile(String id, Map<UUID, Map<String, Long>> counterDump) throws Exception {
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Profile: " + id + ": " + counterDump);
-        }
-        jobManager.reportProfile(id, counterDump);
-    }
-
-    private Handler getRestAPIHandler() {
-        ContextHandler handler = new ContextHandler("/state");
-        RoutingHandler rh = new RoutingHandler();
-        rh.addHandler("jobs", new JSONOutputRequestHandler(new IJSONOutputFunction() {
-            @Override
-            public JSONObject invoke(String[] arguments) throws Exception {
-                JSONObject result = new JSONObject();
-                switch (arguments.length) {
-                    case 1:
-                        if (!"".equals(arguments[0])) {
-                            break;
-                        }
-                    case 0:
-                        result.put("result", jobManager.getQueryInterface().getAllJobSummaries());
-                        break;
-
-                    case 2:
-                        UUID jobId = UUID.fromString(arguments[0]);
-
-                        if ("spec".equalsIgnoreCase(arguments[1])) {
-                            result.put("result", jobManager.getQueryInterface().getJobSpecification(jobId));
-                        } else if ("profile".equalsIgnoreCase(arguments[1])) {
-                            result.put("result", jobManager.getQueryInterface().getJobProfile(jobId));
-                        }
-                }
-                return result;
-            }
-        }));
-        handler.setHandler(rh);
-        return handler;
-    }
-
-    private Handler getAdminConsoleHandler() {
-        ContextHandler handler = new ContextHandler("/admin");
-        handler.setHandler(new AbstractHandler() {
-            @Override
-            public void handle(String target, Request baseRequest, HttpServletRequest request,
-                    HttpServletResponse response) throws IOException, ServletException {
-                if (!"/".equals(target)) {
-                    return;
-                }
-                response.setContentType("text/html;charset=utf-8");
-                response.setStatus(HttpServletResponse.SC_OK);
-                baseRequest.setHandled(true);
-                PrintWriter writer = response.getWriter();
-                writer.println("<html><head><title>Hyracks Admin Console</title></head><body>");
-                writer.println("<h1>Hyracks Admin Console</h1>");
-                writer.println("<h2>Node Controllers</h2>");
-                writer.println("<table><tr><td>Node Id</td><td>Host</td></tr>");
-                synchronized (ClusterControllerService.this) {
-                    for (Map.Entry<String, NodeControllerState> e : nodeRegistry.entrySet()) {
-                        try {
-                            writer.print("<tr><td>");
-                            writer.print(e.getKey());
-                            writer.print("</td><td>");
-                            writer.print(e.getValue().getLastHeartbeatDuration());
-                            writer.print("</td></tr>");
-                        } catch (Exception ex) {
-                        }
-                    }
-                }
-                writer.println("</table>");
-                writer.println("</body></html>");
-                writer.flush();
-            }
-        });
-        return handler;
-    }
-
-    private Handler getApplicationInstallationHandler() {
-        ContextHandler handler = new ContextHandler("/applications");
-        handler.setHandler(new AbstractHandler() {
-            @Override
-            public void handle(String target, Request baseRequest, HttpServletRequest request,
-                    HttpServletResponse response) throws IOException, ServletException {
-                try {
-                    while (target.startsWith("/")) {
-                        target = target.substring(1);
-                    }
-                    while (target.endsWith("/")) {
-                        target = target.substring(0, target.length() - 1);
-                    }
-                    String[] parts = target.split("/");
-                    if (parts.length != 1) {
-                        return;
-                    }
-                    String appName = parts[0];
-                    ApplicationContext appCtx;
-                    appCtx = getApplicationContext(appName);
-                    if (appCtx != null) {
-                        if (HttpMethods.PUT.equals(request.getMethod())) {
-                            OutputStream os = appCtx.getHarOutputStream();
-                            try {
-                                IOUtils.copyLarge(request.getInputStream(), os);
-                            } finally {
-                                os.close();
-                            }
-                        } else if (HttpMethods.GET.equals(request.getMethod())) {
-                            if (!appCtx.containsHar()) {
-                                response.setStatus(HttpServletResponse.SC_NOT_FOUND);
-                            } else {
-                                InputStream is = appCtx.getHarInputStream();
-                                response.setContentType("application/octet-stream");
-                                response.setStatus(HttpServletResponse.SC_OK);
-                                try {
-                                    IOUtils.copyLarge(is, response.getOutputStream());
-                                } finally {
-                                    is.close();
-                                }
-                            }
-                        }
-                        baseRequest.setHandled(true);
-                    }
-                } catch (IOException e) {
-                    e.printStackTrace();
-                    throw e;
-                }
-            }
-        });
-        return handler;
+    public void reportProfile(String id, List<JobProfile> profiles) throws Exception {
+        jobQueue.schedule(new ReportProfilesEvent(this, profiles));
     }
 
     @Override
@@ -375,15 +233,7 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Heartbeat from: " + id);
         }
-        NodeControllerState ns = nodeRegistry.get(id);
-        if (ns != null) {
-            ns.notifyHeartbeat();
-        }
-    }
-
-    private void killNode(String nodeId) throws Exception {
-        nodeRegistry.remove(nodeId);
-        jobManager.notifyNodeFailure(nodeId);
+        jobQueue.schedule(new NodeHeartbeatEvent(this, id));
     }
 
     @Override
@@ -399,44 +249,16 @@
 
     @Override
     public void destroyApplication(String appName) throws Exception {
-        ApplicationContext appCtx = applications.remove(appName);
-        if (appCtx != null) {
-            synchronized (this) {
-                for (NodeControllerState ncs : nodeRegistry.values()) {
-                    ncs.getNodeController().destroyApplication(appName);
-                }
-            }
-            appCtx.deinitialize();
-        }
+        FutureValue fv = new FutureValue();
+        jobQueue.schedule(new ApplicationDestroyEvent(this, appName, fv));
+        fv.get();
     }
 
     @Override
     public void startApplication(final String appName) throws Exception {
-        ApplicationContext appCtx = getApplicationContext(appName);
-        appCtx.initializeClassPath();
-        appCtx.initialize();
-        final byte[] serializedDistributedState = JavaSerializationUtils.serialize(appCtx.getDestributedState());
-        final boolean deployHar = appCtx.containsHar();
-        RemoteOp<Void>[] ops;
-        synchronized (this) {
-            List<RemoteOp<Void>> opList = new ArrayList<RemoteOp<Void>>();
-            for (final String nodeId : nodeRegistry.keySet()) {
-                opList.add(new RemoteOp<Void>() {
-                    @Override
-                    public String getNodeId() {
-                        return nodeId;
-                    }
-
-                    @Override
-                    public Void execute(INodeController node) throws Exception {
-                        node.createApplication(appName, deployHar, serializedDistributedState);
-                        return null;
-                    }
-                });
-            }
-            ops = opList.toArray(new RemoteOp[opList.size()]);
-        }
-        runRemote(ops, null);
+        FutureValue fv = new FutureValue();
+        jobQueue.schedule(new ApplicationStartEvent(this, appName, fv));
+        fv.get();
     }
 
     @Override
@@ -447,282 +269,7 @@
     private class DeadNodeSweeper extends TimerTask {
         @Override
         public void run() {
-            Set<String> deadNodes = new HashSet<String>();
-            synchronized (ClusterControllerService.this) {
-                for (Map.Entry<String, NodeControllerState> e : nodeRegistry.entrySet()) {
-                    NodeControllerState state = e.getValue();
-                    if (state.incrementLastHeartbeatDuration() >= ccConfig.maxHeartbeatLapsePeriods) {
-                        deadNodes.add(e.getKey());
-                    }
-                }
-                for (String deadNode : deadNodes) {
-                    try {
-                        if (LOGGER.isLoggable(Level.INFO)) {
-                            LOGGER.info("Killing node: " + deadNode);
-                        }
-                        killNode(deadNode);
-                    } catch (Exception e) {
-                        e.printStackTrace();
-                    }
-                }
-            }
-        }
-    }
-
-    interface RemoteOp<T> {
-        public String getNodeId();
-
-        public T execute(INodeController node) throws Exception;
-    }
-
-    interface Accumulator<T, R> {
-        public void accumulate(T o);
-
-        public R getResult();
-    }
-
-    <T, R> R runRemote(final RemoteOp<T>[] remoteOps, final Accumulator<T, R> accumulator) throws Exception {
-        final Semaphore installComplete = new Semaphore(remoteOps.length);
-        final List<Exception> errors = new Vector<Exception>();
-        for (final RemoteOp<T> remoteOp : remoteOps) {
-            NodeControllerState nodeState = lookupNode(remoteOp.getNodeId());
-            final INodeController node = nodeState.getNodeController();
-
-            installComplete.acquire();
-            Runnable remoteRunner = new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        T t = remoteOp.execute(node);
-                        if (accumulator != null) {
-                            synchronized (accumulator) {
-                                accumulator.accumulate(t);
-                            }
-                        }
-                    } catch (Exception e) {
-                        errors.add(e);
-                    } finally {
-                        installComplete.release();
-                    }
-                }
-            };
-
-            getExecutor().execute(remoteRunner);
-        }
-        installComplete.acquire(remoteOps.length);
-        if (!errors.isEmpty()) {
-            throw errors.get(0);
-        }
-        return accumulator == null ? null : accumulator.getResult();
-    }
-
-    private static byte[] serialize(Object o) throws IOException {
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        ObjectOutputStream oos = new ObjectOutputStream(baos);
-        oos.writeObject(o);
-        return baos.toByteArray();
-    }
-
-    static 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, Set<Integer>> opPartitions;
-
-        public Phase1Installer(String nodeId, UUID jobId, String appName, JobPlan plan, UUID stageId, int attempt,
-                Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Set<Integer>> opPartitions) {
-            this.nodeId = nodeId;
-            this.jobId = jobId;
-            this.appName = appName;
-            this.plan = plan;
-            this.stageId = stageId;
-            this.attempt = attempt;
-            this.tasks = tasks;
-            this.opPartitions = opPartitions;
-        }
-
-        @Override
-        public Map<PortInstanceId, Endpoint> execute(INodeController node) throws Exception {
-            return node.initializeJobletPhase1(appName, jobId, attempt, serialize(plan), stageId, tasks, opPartitions);
-        }
-
-        @Override
-        public String toString() {
-            return jobId + " Distribution Phase 1";
-        }
-
-        @Override
-        public String getNodeId() {
-            return nodeId;
-        }
-    }
-
-    static 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, Set<Integer>> opPartitions;
-        private Map<PortInstanceId, Endpoint> globalPortMap;
-
-        public Phase2Installer(String nodeId, UUID jobId, String appName, JobPlan plan, UUID stageId,
-                Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Set<Integer>> opPartitions,
-                Map<PortInstanceId, Endpoint> globalPortMap) {
-            this.nodeId = nodeId;
-            this.jobId = jobId;
-            this.appName = appName;
-            this.plan = plan;
-            this.stageId = stageId;
-            this.tasks = tasks;
-            this.opPartitions = opPartitions;
-            this.globalPortMap = globalPortMap;
-        }
-
-        @Override
-        public Void execute(INodeController node) throws Exception {
-            node.initializeJobletPhase2(appName, jobId, serialize(plan), stageId, tasks, opPartitions, globalPortMap);
-            return null;
-        }
-
-        @Override
-        public String toString() {
-            return jobId + " Distribution Phase 2";
-        }
-
-        @Override
-        public String getNodeId() {
-            return nodeId;
-        }
-    }
-
-    static 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;
-        }
-    }
-
-    static 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;
-        }
-    }
-
-    static class JobletAborter implements RemoteOp<Void> {
-        private String nodeId;
-        private UUID jobId;
-        private UUID stageId;
-
-        public JobletAborter(String nodeId, UUID jobId, UUID stageId, int attempt) {
-            this.nodeId = nodeId;
-            this.jobId = jobId;
-            this.stageId = stageId;
-        }
-
-        @Override
-        public Void execute(INodeController node) throws Exception {
-            node.abortJoblet(jobId, stageId);
-            return null;
-        }
-
-        @Override
-        public String toString() {
-            return jobId + " Aborting";
-        }
-
-        @Override
-        public String getNodeId() {
-            return nodeId;
-        }
-    }
-
-    static class JobCompleteNotifier implements RemoteOp<Void> {
-        private String nodeId;
-        private UUID jobId;
-
-        public JobCompleteNotifier(String nodeId, UUID jobId) {
-            this.nodeId = nodeId;
-            this.jobId = jobId;
-        }
-
-        @Override
-        public Void execute(INodeController node) throws Exception {
-            node.cleanUpJob(jobId);
-            return null;
-        }
-
-        @Override
-        public String toString() {
-            return jobId + " Cleaning Up";
-        }
-
-        @Override
-        public String getNodeId() {
-            return nodeId;
-        }
-    }
-
-    static class PortMapMergingAccumulator implements
-            Accumulator<Map<PortInstanceId, Endpoint>, Map<PortInstanceId, Endpoint>> {
-        Map<PortInstanceId, Endpoint> portMap = new HashMap<PortInstanceId, Endpoint>();
-
-        @Override
-        public void accumulate(Map<PortInstanceId, Endpoint> o) {
-            portMap.putAll(o);
-        }
-
-        @Override
-        public Map<PortInstanceId, Endpoint> getResult() {
-            return portMap;
+            jobQueue.schedule(new RemoveDeadNodesEvent(ClusterControllerService.this));
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JOLJobManagerImpl.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JOLJobManagerImpl.java
deleted file mode 100644
index 2eabaf1..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JOLJobManagerImpl.java
+++ /dev/null
@@ -1,1181 +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;
-
-import java.util.ArrayList;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.logging.Logger;
-
-import jol.core.Runtime;
-import jol.types.basic.BasicTupleSet;
-import jol.types.basic.Tuple;
-import jol.types.basic.TupleSet;
-import jol.types.exception.BadKeyException;
-import jol.types.exception.UpdateException;
-import jol.types.table.BasicTable;
-import jol.types.table.EventTable;
-import jol.types.table.Function;
-import jol.types.table.Key;
-import jol.types.table.TableName;
-
-import org.json.JSONArray;
-import org.json.JSONObject;
-
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ChoiceLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionCountConstraint;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.Direction;
-import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
-import edu.uci.ics.hyracks.api.job.JobFlag;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-
-public class JOLJobManagerImpl implements IJobManager {
-    private static final Logger LOGGER = Logger.getLogger(JOLJobManagerImpl.class.getName());
-
-    public static final String JOL_SCOPE = "hyrackscc";
-
-    private static final String SCHEDULER_OLG_FILE = "edu/uci/ics/hyracks/control/cc/scheduler.olg";
-
-    private final Runtime jolRuntime;
-
-    private final LinkedBlockingQueue<Runnable> jobQueue;
-
-    private final JobTable jobTable;
-
-    private final JobQueueThread jobQueueThread;
-
-    private final OperatorDescriptorTable odTable;
-
-    private final OperatorLocationTable olTable;
-
-    private final OperatorCloneCountTable ocTable;
-
-    private final ConnectorDescriptorTable cdTable;
-
-    private final ActivityNodeTable anTable;
-
-    private final ActivityConnectionTable acTable;
-
-    private final ActivityBlockedTable abTable;
-
-    private final JobStartTable jobStartTable;
-
-    private final JobCleanUpTable jobCleanUpTable;
-
-    private final JobCleanUpCompleteTable jobCleanUpCompleteTable;
-
-    private final StartMessageTable startMessageTable;
-
-    private final StageletCompleteTable stageletCompleteTable;
-
-    private final StageletFailureTable stageletFailureTable;
-
-    private final AvailableNodesTable availableNodesTable;
-
-    private final RankedAvailableNodesTable rankedAvailableNodesTable;
-
-    private final FailedNodesTable failedNodesTable;
-
-    private final AbortMessageTable abortMessageTable;
-
-    private final AbortNotifyTable abortNotifyTable;
-
-    private final ExpandPartitionCountConstraintTableFunction expandPartitionCountConstraintFunction;
-
-    private final ProfileUpdateTable puTable;
-
-    private final List<String> rankedAvailableNodes;
-
-    private final IJobManagerQueryInterface qi;
-
-    public JOLJobManagerImpl(final ClusterControllerService ccs, final Runtime jolRuntime) throws Exception {
-        this.jolRuntime = jolRuntime;
-        jobQueue = new LinkedBlockingQueue<Runnable>();
-        jobQueueThread = new JobQueueThread();
-        jobQueueThread.start();
-
-        this.jobTable = new JobTable(jolRuntime);
-        this.odTable = new OperatorDescriptorTable(jolRuntime);
-        this.olTable = new OperatorLocationTable(jolRuntime);
-        this.ocTable = new OperatorCloneCountTable(jolRuntime);
-        this.cdTable = new ConnectorDescriptorTable(jolRuntime);
-        this.anTable = new ActivityNodeTable(jolRuntime);
-        this.acTable = new ActivityConnectionTable(jolRuntime);
-        this.abTable = new ActivityBlockedTable(jolRuntime);
-        this.jobStartTable = new JobStartTable();
-        this.jobCleanUpTable = new JobCleanUpTable(jolRuntime);
-        this.jobCleanUpCompleteTable = new JobCleanUpCompleteTable();
-        this.startMessageTable = new StartMessageTable(jolRuntime);
-        this.stageletCompleteTable = new StageletCompleteTable(jolRuntime);
-        this.stageletFailureTable = new StageletFailureTable(jolRuntime);
-        this.availableNodesTable = new AvailableNodesTable(jolRuntime);
-        this.rankedAvailableNodesTable = new RankedAvailableNodesTable(jolRuntime);
-        this.failedNodesTable = new FailedNodesTable(jolRuntime);
-        this.abortMessageTable = new AbortMessageTable(jolRuntime);
-        this.abortNotifyTable = new AbortNotifyTable(jolRuntime);
-        this.expandPartitionCountConstraintFunction = new ExpandPartitionCountConstraintTableFunction();
-        this.puTable = new ProfileUpdateTable();
-        this.rankedAvailableNodes = new ArrayList<String>();
-
-        jolRuntime.catalog().register(jobTable);
-        jolRuntime.catalog().register(odTable);
-        jolRuntime.catalog().register(olTable);
-        jolRuntime.catalog().register(ocTable);
-        jolRuntime.catalog().register(cdTable);
-        jolRuntime.catalog().register(anTable);
-        jolRuntime.catalog().register(acTable);
-        jolRuntime.catalog().register(abTable);
-        jolRuntime.catalog().register(jobStartTable);
-        jolRuntime.catalog().register(jobCleanUpTable);
-        jolRuntime.catalog().register(jobCleanUpCompleteTable);
-        jolRuntime.catalog().register(startMessageTable);
-        jolRuntime.catalog().register(stageletCompleteTable);
-        jolRuntime.catalog().register(stageletFailureTable);
-        jolRuntime.catalog().register(availableNodesTable);
-        jolRuntime.catalog().register(rankedAvailableNodesTable);
-        jolRuntime.catalog().register(failedNodesTable);
-        jolRuntime.catalog().register(abortMessageTable);
-        jolRuntime.catalog().register(abortNotifyTable);
-        jolRuntime.catalog().register(expandPartitionCountConstraintFunction);
-        jolRuntime.catalog().register(puTable);
-
-        jobTable.register(new JobTable.Callback() {
-            @Override
-            public void deletion(TupleSet arg0) {
-                jobTable.notifyAll();
-            }
-
-            @Override
-            public void insertion(TupleSet arg0) {
-                jobTable.notifyAll();
-            }
-        });
-
-        startMessageTable.register(new StartMessageTable.Callback() {
-            @Override
-            public void deletion(TupleSet tuples) {
-
-            }
-
-            @SuppressWarnings("unchecked")
-            @Override
-            public void insertion(TupleSet tuples) {
-                for (final Tuple t : tuples) {
-                    jobQueue.add(new Runnable() {
-                        @Override
-                        public void run() {
-                            try {
-                                Object[] data = t.toArray();
-                                UUID jobId = (UUID) data[0];
-                                UUID stageId = (UUID) data[1];
-                                Integer attempt = (Integer) data[2];
-                                String appName = (String) data[3];
-                                JobPlan plan = (JobPlan) data[4];
-                                Set<List> ts = (Set<List>) data[5];
-                                Map<OperatorDescriptorId, Set<Integer>> opPartitions = new HashMap<OperatorDescriptorId, Set<Integer>>();
-                                for (List t2 : ts) {
-                                    Object[] t2Data = t2.toArray();
-                                    Set<List> activityInfoSet = (Set<List>) t2Data[1];
-                                    for (List l : activityInfoSet) {
-                                        Object[] lData = l.toArray();
-                                        ActivityNodeId aid = (ActivityNodeId) lData[0];
-                                        Set<Integer> opParts = opPartitions.get(aid.getOperatorDescriptorId());
-                                        if (opParts == null) {
-                                            opParts = new HashSet<Integer>();
-                                            opPartitions.put(aid.getOperatorDescriptorId(), opParts);
-                                        }
-                                        opParts.add((Integer) lData[1]);
-                                    }
-                                }
-                                ClusterControllerService.Phase1Installer[] p1is = new ClusterControllerService.Phase1Installer[ts
-                                        .size()];
-                                int i = 0;
-                                for (List t2 : ts) {
-                                    Object[] t2Data = t2.toArray();
-                                    Map<ActivityNodeId, Set<Integer>> tasks = new HashMap<ActivityNodeId, Set<Integer>>();
-                                    Set<List> activityInfoSet = (Set<List>) t2Data[1];
-                                    for (List l : activityInfoSet) {
-                                        Object[] lData = l.toArray();
-                                        ActivityNodeId aid = (ActivityNodeId) lData[0];
-                                        Set<Integer> aParts = tasks.get(aid);
-                                        if (aParts == null) {
-                                            aParts = new HashSet<Integer>();
-                                            tasks.put(aid, aParts);
-                                        }
-                                        aParts.add((Integer) lData[1]);
-                                    }
-                                    p1is[i++] = new ClusterControllerService.Phase1Installer((String) t2Data[0], jobId,
-                                            appName, plan, stageId, attempt, tasks, opPartitions);
-                                }
-                                LOGGER.info("Stage start - Phase 1");
-                                Map<PortInstanceId, Endpoint> globalPortMap = ccs.runRemote(p1is,
-                                        new ClusterControllerService.PortMapMergingAccumulator());
-
-                                ClusterControllerService.Phase2Installer[] p2is = new ClusterControllerService.Phase2Installer[ts
-                                        .size()];
-                                ClusterControllerService.Phase3Installer[] p3is = new ClusterControllerService.Phase3Installer[ts
-                                        .size()];
-                                ClusterControllerService.StageStarter[] ss = new ClusterControllerService.StageStarter[ts
-                                        .size()];
-                                i = 0;
-                                for (List t2 : ts) {
-                                    Object[] t2Data = t2.toArray();
-                                    Map<ActivityNodeId, Set<Integer>> tasks = new HashMap<ActivityNodeId, Set<Integer>>();
-                                    Set<List> activityInfoSet = (Set<List>) t2Data[1];
-                                    for (List l : activityInfoSet) {
-                                        Object[] lData = l.toArray();
-                                        ActivityNodeId aid = (ActivityNodeId) lData[0];
-                                        Set<Integer> aParts = tasks.get(aid);
-                                        if (aParts == null) {
-                                            aParts = new HashSet<Integer>();
-                                            tasks.put(aid, aParts);
-                                        }
-                                        aParts.add((Integer) lData[1]);
-                                    }
-                                    p2is[i] = new ClusterControllerService.Phase2Installer((String) t2Data[0], jobId,
-                                            appName, plan, stageId, tasks, opPartitions, globalPortMap);
-                                    p3is[i] = new ClusterControllerService.Phase3Installer((String) t2Data[0], jobId,
-                                            stageId);
-                                    ss[i] = new ClusterControllerService.StageStarter((String) t2Data[0], jobId,
-                                            stageId);
-                                    ++i;
-                                }
-                                LOGGER.info("Stage start - Phase 2");
-                                ccs.runRemote(p2is, null);
-                                LOGGER.info("Stage start - Phase 3");
-                                ccs.runRemote(p3is, null);
-                                LOGGER.info("Stage start");
-                                ccs.runRemote(ss, null);
-                                LOGGER.info("Stage started");
-                            } catch (Exception e) {
-                            }
-                        }
-                    });
-                }
-            }
-        });
-
-        jobCleanUpTable.register(new JobCleanUpTable.Callback() {
-            @Override
-            public void deletion(TupleSet tuples) {
-            }
-
-            @SuppressWarnings("unchecked")
-            @Override
-            public void insertion(TupleSet tuples) {
-                for (final Tuple t : tuples) {
-                    jobQueue.add(new Runnable() {
-                        @Override
-                        public void run() {
-                            try {
-                                Object[] data = t.toArray();
-                                UUID jobId = (UUID) data[0];
-                                String appName = (String) data[1];
-                                Set<String> ts = (Set<String>) data[2];
-                                ClusterControllerService.JobCompleteNotifier[] jcns = new ClusterControllerService.JobCompleteNotifier[ts
-                                        .size()];
-                                int i = 0;
-                                for (String n : ts) {
-                                    jcns[i++] = new ClusterControllerService.JobCompleteNotifier(n, jobId);
-                                }
-                                try {
-                                    ccs.runRemote(jcns, null);
-                                } finally {
-                                    BasicTupleSet jccTuples = new BasicTupleSet(JobCleanUpCompleteTable
-                                            .createTuple(jobId));
-                                    jolRuntime.schedule(JOL_SCOPE, JobCleanUpCompleteTable.TABLE_NAME, jccTuples, null);
-                                    jolRuntime.evaluate();
-                                }
-                                CCApplicationContext appCtx = ccs.getApplicationContext(appName);
-                                if (appCtx != null) {
-                                    appCtx.notifyJobFinish(jobId);
-                                }
-                            } catch (Exception e) {
-                            }
-                        }
-                    });
-                }
-            }
-        });
-
-        abortMessageTable.register(new AbortMessageTable.Callback() {
-            @Override
-            public void deletion(TupleSet tuples) {
-
-            }
-
-            @SuppressWarnings("unchecked")
-            @Override
-            public void insertion(TupleSet tuples) {
-                for (final Tuple t : tuples) {
-                    jobQueue.add(new Runnable() {
-                        @Override
-                        public void run() {
-                            try {
-                                Object[] data = t.toArray();
-                                UUID jobId = (UUID) data[0];
-                                UUID stageId = (UUID) data[1];
-                                Integer attempt = (Integer) data[2];
-                                Set<List> ts = (Set<List>) data[4];
-                                ClusterControllerService.JobletAborter[] jas = new ClusterControllerService.JobletAborter[ts
-                                        .size()];
-                                int i = 0;
-                                BasicTupleSet notificationTuples = new BasicTupleSet();
-                                for (List t2 : ts) {
-                                    Object[] t2Data = t2.toArray();
-                                    String nodeId = (String) t2Data[0];
-                                    jas[i++] = new ClusterControllerService.JobletAborter(nodeId, jobId, stageId,
-                                            attempt);
-                                    notificationTuples.add(AbortNotifyTable
-                                            .createTuple(jobId, stageId, nodeId, attempt));
-                                }
-                                try {
-                                    ccs.runRemote(jas, null);
-                                } finally {
-                                    jolRuntime.schedule(JOL_SCOPE, AbortNotifyTable.TABLE_NAME, notificationTuples,
-                                            null);
-                                    jolRuntime.evaluate();
-                                }
-                            } catch (Exception e) {
-                            }
-                        }
-                    });
-                }
-            }
-        });
-
-        jolRuntime.install(JOL_SCOPE, ClassLoader.getSystemResource(SCHEDULER_OLG_FILE));
-        jolRuntime.evaluate();
-
-        qi = new QueryInterfaceImpl();
-    }
-
-    @Override
-    public void createJob(final UUID jobId, String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags)
-            throws Exception {
-        final JobPlanBuilder builder = new JobPlanBuilder();
-        builder.init(jobSpec, jobFlags);
-
-        final BasicTupleSet anTuples = new BasicTupleSet();
-        final BasicTupleSet acTuples = new BasicTupleSet();
-        final BasicTupleSet abTuples = new BasicTupleSet();
-        IActivityGraphBuilder gBuilder = new IActivityGraphBuilder() {
-            @Override
-            public void addTask(IActivityNode task) {
-                anTuples.add(ActivityNodeTable.createTuple(jobId, task));
-                builder.addTask(task);
-            }
-
-            @Override
-            public void addTargetEdge(int operatorOutputIndex, IActivityNode task, int taskOutputIndex) {
-                acTuples.add(ActivityConnectionTable.createTuple(jobId, task, Direction.OUTPUT, operatorOutputIndex,
-                        taskOutputIndex));
-                builder.addTargetEdge(operatorOutputIndex, task, taskOutputIndex);
-            }
-
-            @Override
-            public void addSourceEdge(int operatorInputIndex, IActivityNode task, int taskInputIndex) {
-                acTuples.add(ActivityConnectionTable.createTuple(jobId, task, Direction.INPUT, operatorInputIndex,
-                        taskInputIndex));
-                builder.addSourceEdge(operatorInputIndex, task, taskInputIndex);
-            }
-
-            @Override
-            public void addBlockingEdge(IActivityNode blocker, IActivityNode blocked) {
-                abTuples.add(ActivityBlockedTable.createTuple(jobId, blocker, blocked));
-                builder.addBlockingEdge(blocker, blocked);
-            }
-        };
-
-        BasicTupleSet odTuples = new BasicTupleSet();
-        BasicTupleSet olTuples = new BasicTupleSet();
-        BasicTupleSet ocTuples = new BasicTupleSet();
-        for (Map.Entry<OperatorDescriptorId, IOperatorDescriptor> e : jobSpec.getOperatorMap().entrySet()) {
-            IOperatorDescriptor od = e.getValue();
-            int nPartitions = addPartitionConstraintTuples(jobId, od, olTuples, ocTuples);
-            odTuples.add(OperatorDescriptorTable.createTuple(jobId, nPartitions, od));
-            od.contributeTaskGraph(gBuilder);
-        }
-
-        BasicTupleSet cdTuples = new BasicTupleSet();
-        for (Map.Entry<ConnectorDescriptorId, IConnectorDescriptor> e : jobSpec.getConnectorMap().entrySet()) {
-            cdTuples.add(ConnectorDescriptorTable.createTuple(jobId, jobSpec, e.getValue()));
-        }
-
-        BasicTupleSet jobTuples = new BasicTupleSet(JobTable.createInitialJobTuple(jobId, appName, jobSpec,
-                builder.getPlan()));
-
-        jolRuntime.schedule(JOL_SCOPE, JobTable.TABLE_NAME, jobTuples, null);
-        jolRuntime.schedule(JOL_SCOPE, OperatorDescriptorTable.TABLE_NAME, odTuples, null);
-        jolRuntime.schedule(JOL_SCOPE, OperatorLocationTable.TABLE_NAME, olTuples, null);
-        jolRuntime.schedule(JOL_SCOPE, OperatorCloneCountTable.TABLE_NAME, ocTuples, null);
-        jolRuntime.schedule(JOL_SCOPE, ConnectorDescriptorTable.TABLE_NAME, cdTuples, null);
-        jolRuntime.schedule(JOL_SCOPE, ActivityNodeTable.TABLE_NAME, anTuples, null);
-        jolRuntime.schedule(JOL_SCOPE, ActivityConnectionTable.TABLE_NAME, acTuples, null);
-        jolRuntime.schedule(JOL_SCOPE, ActivityBlockedTable.TABLE_NAME, abTuples, null);
-
-        jolRuntime.evaluate();
-    }
-
-    private int addPartitionConstraintTuples(UUID jobId, IOperatorDescriptor od, BasicTupleSet olTuples,
-            BasicTupleSet ocTuples) {
-        PartitionConstraint pc = od.getPartitionConstraint();
-
-        switch (pc.getPartitionConstraintType()) {
-            case COUNT:
-                int count = ((PartitionCountConstraint) pc).getCount();
-                ocTuples.add(OperatorCloneCountTable.createTuple(jobId, od.getOperatorId(), count));
-                return count;
-
-            case EXPLICIT:
-                LocationConstraint[] locationConstraints = ((ExplicitPartitionConstraint) pc).getLocationConstraints();
-                for (int i = 0; i < locationConstraints.length; ++i) {
-                    addLocationConstraintTuple(olTuples, jobId, od.getOperatorId(), i, locationConstraints[i], 0);
-                }
-                return locationConstraints.length;
-        }
-        throw new IllegalArgumentException();
-    }
-
-    private void addLocationConstraintTuple(BasicTupleSet olTuples, UUID jobId, OperatorDescriptorId opId, int i,
-            LocationConstraint locationConstraint, int benefit) {
-        switch (locationConstraint.getConstraintType()) {
-            case ABSOLUTE:
-                String nodeId = ((AbsoluteLocationConstraint) locationConstraint).getLocationId();
-                olTuples.add(OperatorLocationTable.createTuple(jobId, opId, nodeId, i, benefit));
-                break;
-
-            case CHOICE:
-                int index = 0;
-                for (LocationConstraint lc : ((ChoiceLocationConstraint) locationConstraint).getChoices()) {
-                    addLocationConstraintTuple(olTuples, jobId, opId, i, lc, benefit - index);
-                    index++;
-                }
-        }
-    }
-
-    @Override
-    public void reportProfile(String id, Map<UUID, Map<String, Long>> counterDump) throws Exception {
-        BasicTupleSet puTuples = new BasicTupleSet();
-        for (Map.Entry<UUID, Map<String, Long>> e : counterDump.entrySet()) {
-            puTuples.add(ProfileUpdateTable.createTuple(e.getKey(), id, e.getValue()));
-        }
-        jolRuntime.schedule(JOL_SCOPE, ProfileUpdateTable.TABLE_NAME, puTuples, null);
-    }
-
-    @Override
-    public JobStatus getJobStatus(UUID jobId) {
-        synchronized (jobTable) {
-            try {
-                Tuple jobTuple = jobTable.lookupJob(jobId);
-                if (jobTuple == null) {
-                    return null;
-                }
-                return (JobStatus) jobTuple.value(JobTable.JOBSTATUS_FIELD_INDEX);
-            } catch (BadKeyException e) {
-                throw new RuntimeException(e);
-            }
-        }
-    }
-
-    @Override
-    public synchronized void notifyNodeFailure(String nodeId) throws Exception {
-        int len = rankedAvailableNodes.size();
-        int delIndex = -1;
-        for (int i = 0; i < len; ++i) {
-            if (nodeId.equals(rankedAvailableNodes.get(i))) {
-                delIndex = i;
-                break;
-            }
-        }
-        if (delIndex < 0) {
-            return;
-        }
-        BasicTupleSet delRANTuples = new BasicTupleSet();
-        delRANTuples.add(RankedAvailableNodesTable.createTuple(nodeId, delIndex));
-
-        BasicTupleSet insRANTuples = new BasicTupleSet();
-        for (int i = delIndex + 1; i < len; ++i) {
-            insRANTuples.add(RankedAvailableNodesTable.createTuple(rankedAvailableNodes.get(i), i - 1));
-        }
-
-        rankedAvailableNodes.remove(delIndex);
-
-        jolRuntime.schedule(JOL_SCOPE, RankedAvailableNodesTable.TABLE_NAME, insRANTuples, delRANTuples);
-
-        BasicTupleSet unavailableTuples = new BasicTupleSet(AvailableNodesTable.createTuple(nodeId));
-
-        jolRuntime.schedule(JOL_SCOPE, AvailableNodesTable.TABLE_NAME, null, unavailableTuples);
-
-        jolRuntime.evaluate();
-
-        BasicTupleSet failedTuples = new BasicTupleSet(FailedNodesTable.createTuple(nodeId));
-
-        jolRuntime.schedule(JOL_SCOPE, FailedNodesTable.TABLE_NAME, failedTuples, null);
-
-        jolRuntime.evaluate();
-    }
-
-    @Override
-    public synchronized void notifyStageletComplete(UUID jobId, UUID stageId, int attempt, String nodeId,
-            Map<String, Long> statistics) throws Exception {
-        BasicTupleSet scTuples = new BasicTupleSet();
-        scTuples.add(StageletCompleteTable.createTuple(jobId, stageId, nodeId, attempt, statistics));
-
-        jolRuntime.schedule(JOL_SCOPE, StageletCompleteTable.TABLE_NAME, scTuples, null);
-
-        jolRuntime.evaluate();
-    }
-
-    @Override
-    public synchronized void notifyStageletFailure(UUID jobId, UUID stageId, int attempt, String nodeId)
-            throws Exception {
-        BasicTupleSet sfTuples = new BasicTupleSet();
-        sfTuples.add(StageletFailureTable.createTuple(jobId, stageId, nodeId, attempt));
-
-        jolRuntime.schedule(JOL_SCOPE, StageletFailureTable.TABLE_NAME, sfTuples, null);
-
-        jolRuntime.evaluate();
-    }
-
-    @Override
-    public void start(UUID jobId) throws Exception {
-        BasicTupleSet jsTuples = new BasicTupleSet();
-        jsTuples.add(JobStartTable.createTuple(jobId, System.currentTimeMillis()));
-
-        jolRuntime.schedule(JOL_SCOPE, JobStartTable.TABLE_NAME, jsTuples, null);
-
-        jolRuntime.evaluate();
-    }
-
-    @Override
-    public synchronized void registerNode(String nodeId) throws Exception {
-        rankedAvailableNodes.add(nodeId);
-        BasicTupleSet insRANTuples = new BasicTupleSet();
-        insRANTuples.add(RankedAvailableNodesTable.createTuple(nodeId, rankedAvailableNodes.size() - 1));
-
-        jolRuntime.schedule(JOL_SCOPE, RankedAvailableNodesTable.TABLE_NAME, insRANTuples, null);
-
-        BasicTupleSet availableTuples = new BasicTupleSet(AvailableNodesTable.createTuple(nodeId));
-
-        jolRuntime.schedule(JOL_SCOPE, AvailableNodesTable.TABLE_NAME, availableTuples, null);
-
-        jolRuntime.evaluate();
-
-        BasicTupleSet unfailedTuples = new BasicTupleSet(FailedNodesTable.createTuple(nodeId));
-
-        jolRuntime.schedule(JOL_SCOPE, FailedNodesTable.TABLE_NAME, null, unfailedTuples);
-
-        jolRuntime.evaluate();
-    }
-
-    @Override
-    public void waitForCompletion(UUID jobId) throws Exception {
-        synchronized (jobTable) {
-            Tuple jobTuple = null;
-            while ((jobTuple = jobTable.lookupJob(jobId)) != null
-                    && jobTuple.value(JobTable.JOBSTATUS_FIELD_INDEX) != JobStatus.TERMINATED) {
-                jobTable.wait();
-            }
-        }
-    }
-
-    @Override
-    public IJobManagerQueryInterface getQueryInterface() {
-        return qi;
-    }
-
-    public void visitJobs(ITupleProcessor processor) throws Exception {
-        for (Tuple t : jobTable.tuples()) {
-            processor.process(t);
-        }
-    }
-
-    public void visitJob(UUID jobId, ITupleProcessor processor) throws Exception {
-        Tuple job = jobTable.lookupJob(jobId);
-        if (job != null) {
-            processor.process(job);
-        }
-    }
-
-    /*
-     * declare(job, keys(0), {JobId, Status, JobSpec, JobPlan})
-     */
-    private static class JobTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "job");
-
-        private static Key PRIMARY_KEY = new Key(0);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, String.class, JobStatus.class,
-                JobSpecification.class, JobPlan.class, Map.class };
-
-        public static final int JOBID_FIELD_INDEX = 0;
-        public static final int APPNAME_FIELD_INDEX = 1;
-        public static final int JOBSTATUS_FIELD_INDEX = 2;
-        public static final int JOBSPEC_FIELD_INDEX = 3;
-        public static final int JOBPLAN_FIELD_INDEX = 4;
-        public static final int STATISTICS_FIELD_INDEX = 5;
-
-        public JobTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        @SuppressWarnings("unchecked")
-        static Tuple createInitialJobTuple(UUID jobId, String appName, JobSpecification jobSpec, JobPlan plan) {
-            return new Tuple(jobId, appName, JobStatus.INITIALIZED, jobSpec, plan, new HashMap());
-        }
-
-        Tuple lookupJob(UUID jobId) throws BadKeyException {
-            TupleSet set = primary().lookupByKey(jobId);
-            if (set.isEmpty()) {
-                return null;
-            }
-            return (Tuple) set.toArray()[0];
-        }
-    }
-
-    /*
-     * declare(operatordescriptor, keys(0, 1), {JobId, ODId, OperatorDescriptor})
-     */
-    private static class OperatorDescriptorTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "operatordescriptor");
-
-        private static Key PRIMARY_KEY = new Key(0, 1);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, OperatorDescriptorId.class, Integer.class,
-                IOperatorDescriptor.class };
-
-        public OperatorDescriptorTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        static Tuple createTuple(UUID jobId, int nPartitions, IOperatorDescriptor od) {
-            return new Tuple(jobId, od.getOperatorId(), nPartitions, od);
-        }
-    }
-
-    /*
-     * declare(operatorlocation, keys(0, 1), {JobId, ODId, NodeId})
-     */
-    private static class OperatorLocationTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "operatorlocation");
-
-        private static Key PRIMARY_KEY = new Key();
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, OperatorDescriptorId.class, String.class,
-                Integer.class, Integer.class };
-
-        public OperatorLocationTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        static Tuple createTuple(UUID jobId, OperatorDescriptorId opId, String nodeId, int partition, int benefit) {
-            return new Tuple(jobId, opId, nodeId, partition, benefit);
-        }
-    }
-
-    /*
-     * declare(operatorclonecount, keys(0, 1), {JobId, ODId, Count})
-     */
-    private static class OperatorCloneCountTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "operatorclonecount");
-
-        private static Key PRIMARY_KEY = new Key();
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, OperatorDescriptorId.class, Integer.class };
-
-        public OperatorCloneCountTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        static Tuple createTuple(UUID jobId, OperatorDescriptorId opId, int cloneCount) {
-            return new Tuple(jobId, opId, cloneCount);
-        }
-    }
-
-    /*
-     * declare(connectordescriptor, keys(0, 1), {JobId, CDId, SrcODId, SrcPort, DestODId, DestPort, ConnectorDescriptor})
-     */
-    private static class ConnectorDescriptorTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "connectordescriptor");
-
-        private static Key PRIMARY_KEY = new Key(0, 1);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, ConnectorDescriptorId.class,
-                OperatorDescriptorId.class, Integer.class, OperatorDescriptorId.class, Integer.class,
-                IConnectorDescriptor.class };
-
-        public ConnectorDescriptorTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        static Tuple createTuple(UUID jobId, JobSpecification jobSpec, IConnectorDescriptor conn) {
-            IOperatorDescriptor srcOD = jobSpec.getProducer(conn);
-            int srcPort = jobSpec.getProducerOutputIndex(conn);
-            IOperatorDescriptor destOD = jobSpec.getConsumer(conn);
-            int destPort = jobSpec.getConsumerInputIndex(conn);
-            Tuple cdTuple = new Tuple(jobId, conn.getConnectorId(), srcOD.getOperatorId(), srcPort,
-                    destOD.getOperatorId(), destPort, conn);
-            return cdTuple;
-        }
-    }
-
-    /*
-     * declare(activitynode, keys(0, 1, 2), {JobId, OperatorId, ActivityId, ActivityNode})
-     */
-    private static class ActivityNodeTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "activitynode");
-
-        private static Key PRIMARY_KEY = new Key(0, 1, 2);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, OperatorDescriptorId.class,
-                ActivityNodeId.class, IActivityNode.class };
-
-        public ActivityNodeTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        static Tuple createTuple(UUID jobId, IActivityNode aNode) {
-            return new Tuple(jobId, aNode.getActivityNodeId().getOperatorDescriptorId(), aNode.getActivityNodeId(),
-                    aNode);
-        }
-    }
-
-    /*
-     * declare(activityconnection, keys(0, 1, 2, 3), {JobId, OperatorId, Integer, Direction, ActivityNodeId, Integer})
-     */
-    private static class ActivityConnectionTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "activityconnection");
-
-        private static Key PRIMARY_KEY = new Key(0, 1, 2, 3);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, OperatorDescriptorId.class, Integer.class,
-                Direction.class, ActivityNodeId.class, Integer.class };
-
-        public ActivityConnectionTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        static Tuple createTuple(UUID jobId, IActivityNode aNode, Direction direction, int odPort, int activityPort) {
-            return new Tuple(jobId, aNode.getActivityNodeId().getOperatorDescriptorId(), odPort, direction,
-                    aNode.getActivityNodeId(), activityPort);
-        }
-    }
-
-    /*
-     * declare(activityblocked, keys(0, 1, 2, 3), {JobId, OperatorId, BlockerActivityId, BlockedActivityId})
-     */
-    private static class ActivityBlockedTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "activityblocked");
-
-        private static Key PRIMARY_KEY = new Key(0, 1, 2, 3, 4);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, OperatorDescriptorId.class,
-                ActivityNodeId.class, OperatorDescriptorId.class, ActivityNodeId.class };
-
-        public ActivityBlockedTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        static Tuple createTuple(UUID jobId, IActivityNode blocker, IActivityNode blocked) {
-            ActivityNodeId blockerANId = blocker.getActivityNodeId();
-            OperatorDescriptorId blockerODId = blockerANId.getOperatorDescriptorId();
-            ActivityNodeId blockedANId = blocked.getActivityNodeId();
-            OperatorDescriptorId blockedODId = blockedANId.getOperatorDescriptorId();
-            return new Tuple(jobId, blockerODId, blockerANId, blockedODId, blockedANId);
-        }
-    }
-
-    /*
-     * declare(jobstart, keys(0), {JobId, SubmitTime})
-     */
-    private static class JobStartTable extends EventTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "jobstart");
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, Long.class };
-
-        public JobStartTable() {
-            super(TABLE_NAME, SCHEMA);
-        }
-
-        static Tuple createTuple(UUID jobId, long submitTime) {
-            return new Tuple(jobId, submitTime);
-        }
-    }
-
-    /*
-     * declare(startmessage, keys(0, 1), {JobId, StageId, JobPlan, TupleSet})
-     */
-    private static class StartMessageTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "startmessage");
-
-        private static Key PRIMARY_KEY = new Key(0, 1, 2);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, UUID.class, Integer.class, String.class,
-                JobPlan.class, Set.class };
-
-        public StartMessageTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-    }
-
-    /*
-     * declare(jobcleanup, keys(0), {JobId, Set<NodeId>})
-     */
-    private static class JobCleanUpTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "jobcleanup");
-
-        private static Key PRIMARY_KEY = new Key(0);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, String.class, Set.class };
-
-        public JobCleanUpTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-    }
-
-    /*
-     * declare(jobcleanupcomplete, keys(0), {JobId})
-     */
-    private static class JobCleanUpCompleteTable extends EventTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "jobcleanupcomplete");
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class };
-
-        public JobCleanUpCompleteTable() {
-            super(TABLE_NAME, SCHEMA);
-        }
-
-        public static Tuple createTuple(UUID jobId) {
-            return new Tuple(jobId);
-        }
-    }
-
-    /*
-     * declare(stageletcomplete, keys(0, 1, 2, 3), {JobId, StageId, NodeId, Attempt})
-     */
-    private static class StageletCompleteTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "stageletcomplete");
-
-        private static Key PRIMARY_KEY = new Key(0, 1, 2, 3);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, UUID.class, String.class, Integer.class,
-                Map.class };
-
-        public StageletCompleteTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        public static Tuple createTuple(UUID jobId, UUID stageId, String nodeId, int attempt,
-                Map<String, Long> statistics) {
-            return new Tuple(jobId, stageId, nodeId, attempt, statistics);
-        }
-    }
-
-    /*
-     * declare(stageletfailure, keys(0, 1, 2, 3), {JobId, StageId, NodeId, Attempt})
-     */
-    private static class StageletFailureTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "stageletfailure");
-
-        private static Key PRIMARY_KEY = new Key(0, 1, 2, 3);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, UUID.class, String.class, Integer.class };
-
-        public StageletFailureTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        public static Tuple createTuple(UUID jobId, UUID stageId, String nodeId, int attempt) {
-            return new Tuple(jobId, stageId, nodeId, attempt);
-        }
-    }
-
-    /*
-     * declare(availablenodes, keys(0), {NodeId})
-     */
-    private static class AvailableNodesTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "availablenodes");
-
-        private static Key PRIMARY_KEY = new Key(0);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { String.class };
-
-        public AvailableNodesTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        public static Tuple createTuple(String nodeId) {
-            return new Tuple(nodeId);
-        }
-    }
-
-    /*
-     * declare(rankedavailablenodes, keys(0), {NodeId, Integer})
-     */
-    private static class RankedAvailableNodesTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "rankedavailablenodes");
-
-        private static Key PRIMARY_KEY = new Key(0);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { String.class, Integer.class };
-
-        public RankedAvailableNodesTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        public static Tuple createTuple(String nodeId, int rank) {
-            return new Tuple(nodeId, rank);
-        }
-    }
-
-    /*
-     * declare(failednodes, keys(0), {NodeId})
-     */
-    private static class FailedNodesTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "failednodes");
-
-        private static Key PRIMARY_KEY = new Key(0);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { String.class };
-
-        public FailedNodesTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        public static Tuple createTuple(String nodeId) {
-            return new Tuple(nodeId);
-        }
-    }
-
-    /*
-     * declare(abortmessage, keys(0, 1), {JobId, StageId, Attempt, JobPlan, TupleSet})
-     */
-    private static class AbortMessageTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "abortmessage");
-
-        private static Key PRIMARY_KEY = new Key(0, 1, 2);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, UUID.class, Integer.class, JobPlan.class,
-                Set.class };
-
-        public AbortMessageTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-    }
-
-    /*
-     * declare(abortnotify, keys(0, 1, 2, 3), {JobId, StageId, NodeId, Attempt, StageletStatistics})
-     */
-    private static class AbortNotifyTable extends BasicTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "abortnotify");
-
-        private static Key PRIMARY_KEY = new Key(0, 1, 2, 3);
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, UUID.class, String.class, Integer.class };
-
-        public AbortNotifyTable(Runtime context) {
-            super(context, TABLE_NAME, PRIMARY_KEY, SCHEMA);
-        }
-
-        public static Tuple createTuple(UUID jobId, UUID stageId, String nodeId, int attempt) {
-            return new Tuple(jobId, stageId, nodeId, attempt);
-        }
-    }
-
-    private static class ExpandPartitionCountConstraintTableFunction extends Function {
-        private static final String TABLE_NAME = "expandpartitioncountconstraint";
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, OperatorDescriptorId.class, Integer.class,
-                Integer.class };
-
-        public static final int JOBID_FIELD_INDEX = 0;
-        public static final int ODID_FIELD_INDEX = 1;
-        public static final int NPARTITIONS_FIELD_INDEX = 2;
-        public static final int ORDINAL_FIELD_INDEX = 3;
-
-        public ExpandPartitionCountConstraintTableFunction() {
-            super(TABLE_NAME, SCHEMA);
-        }
-
-        @Override
-        public TupleSet insert(TupleSet tuples, TupleSet conflicts) throws UpdateException {
-            TupleSet result = new BasicTupleSet();
-            int counter = 0;
-            for (Tuple t : tuples) {
-                int nPartitions = (Integer) t.value(NPARTITIONS_FIELD_INDEX);
-                for (int i = 0; i < nPartitions; ++i) {
-                    result.add(new Tuple(t.value(JOBID_FIELD_INDEX), t.value(ODID_FIELD_INDEX), i, counter++));
-                }
-            }
-            return result;
-        }
-    }
-
-    /*
-     * declare(profileupdate, keys(0, 1), {JobId, NodeId, Map})
-     */
-    private static class ProfileUpdateTable extends EventTable {
-        private static TableName TABLE_NAME = new TableName(JOL_SCOPE, "profileupdate");
-
-        @SuppressWarnings("unchecked")
-        private static final Class[] SCHEMA = new Class[] { UUID.class, String.class, Map.class };
-
-        public ProfileUpdateTable() {
-            super(TABLE_NAME, SCHEMA);
-        }
-
-        public static Tuple createTuple(UUID jobId, String nodeId, Map<String, Long> statistics) {
-            return new Tuple(jobId, nodeId, statistics);
-        }
-    }
-
-    private class JobQueueThread extends Thread {
-        public JobQueueThread() {
-            setDaemon(true);
-        }
-
-        public void run() {
-            Runnable r;
-            while (true) {
-                try {
-                    r = jobQueue.take();
-                } catch (InterruptedException e) {
-                    continue;
-                }
-                try {
-                    r.run();
-                } catch (Exception e) {
-                    e.printStackTrace();
-                }
-            }
-        }
-    }
-
-    public interface ITupleProcessor {
-        public void process(Tuple t) throws Exception;
-    }
-
-    private class QueryInterfaceImpl implements IJobManagerQueryInterface {
-        @Override
-        public JSONArray getAllJobSummaries() throws Exception {
-            final JSONArray jobs = new JSONArray();
-            JOLJobManagerImpl.ITupleProcessor tp = new JOLJobManagerImpl.ITupleProcessor() {
-                @Override
-                public void process(Tuple t) throws Exception {
-                    JSONObject jo = new JSONObject();
-                    jo.put("type", "job-summary");
-                    jo.put("id", t.value(JobTable.JOBID_FIELD_INDEX).toString());
-                    jo.put("status", t.value(JobTable.JOBSTATUS_FIELD_INDEX).toString());
-                    jobs.put(jo);
-                }
-            };
-            visitJobs(tp);
-            return jobs;
-        }
-
-        @Override
-        public JSONObject getJobSpecification(UUID jobId) throws Exception {
-            final JSONArray jobs = new JSONArray();
-            JOLJobManagerImpl.ITupleProcessor tp = new JOLJobManagerImpl.ITupleProcessor() {
-                @Override
-                public void process(Tuple t) throws Exception {
-                    JobSpecification js = (JobSpecification) t.value(JobTable.JOBSPEC_FIELD_INDEX);
-                    jobs.put(js.toJSON());
-                }
-            };
-            visitJob(jobId, tp);
-            return (JSONObject) (jobs.length() == 0 ? new JSONObject() : jobs.get(0));
-        }
-
-        @Override
-        public JSONObject getJobPlan(UUID jobId) throws Exception {
-            final JSONArray jobs = new JSONArray();
-            JOLJobManagerImpl.ITupleProcessor tp = new JOLJobManagerImpl.ITupleProcessor() {
-                @Override
-                public void process(Tuple t) throws Exception {
-                }
-            };
-            visitJob(jobId, tp);
-            return (JSONObject) (jobs.length() == 0 ? new JSONObject() : jobs.get(0));
-        }
-
-        @Override
-        public JSONObject getJobProfile(UUID jobId) throws Exception {
-            final JSONArray jobs = new JSONArray();
-            JOLJobManagerImpl.ITupleProcessor tp = new JOLJobManagerImpl.ITupleProcessor() {
-                @Override
-                public void process(Tuple t) throws Exception {
-                    JSONObject jo = new JSONObject();
-                    jo.put("type", "profile");
-                    jo.put("id", t.value(JobTable.JOBID_FIELD_INDEX).toString());
-                    Map<String, Long> profile = (Map<String, Long>) t.value(JobTable.STATISTICS_FIELD_INDEX);
-                    if (profile != null) {
-                        for (Map.Entry<String, Long> e : profile.entrySet()) {
-                            JSONObject jpe = new JSONObject();
-                            jpe.put("name", e.getKey());
-                            jpe.put("value", e.getValue());
-                            jo.accumulate("counters", jpe);
-                        }
-                    }
-                    jobs.put(jo);
-                }
-            };
-            visitJob(jobId, tp);
-            return (JSONObject) (jobs.length() == 0 ? new JSONObject() : jobs.get(0));
-        }
-    }
-}
\ No newline at end of file
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 a638c09..e69281b 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,29 +1,40 @@
 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;
 
 public class NodeControllerState {
     private final INodeController nodeController;
 
+    private final Set<UUID> activeJobIds;
+
     private int lastHeartbeatDuration;
 
     public NodeControllerState(INodeController nodeController) {
         this.nodeController = nodeController;
+        activeJobIds = new HashSet<UUID>();
     }
 
-    void notifyHeartbeat() {
+    public void notifyHeartbeat() {
         lastHeartbeatDuration = 0;
     }
 
-    int incrementLastHeartbeatDuration() {
+    public int incrementLastHeartbeatDuration() {
         return lastHeartbeatDuration++;
     }
 
-    int getLastHeartbeatDuration() {
+    public int getLastHeartbeatDuration() {
         return lastHeartbeatDuration;
     }
 
     public INodeController getNodeController() {
         return nodeController;
     }
+
+    public Set<UUID> getActiveJobIds() {
+        return activeJobIds;
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/IConnectorDescriptorVisitor.java
similarity index 68%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
copy to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/IConnectorDescriptorVisitor.java
index 75edf2b..1c8446f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/IConnectorDescriptorVisitor.java
@@ -12,10 +12,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.web.handlers.util;
+package edu.uci.ics.hyracks.control.cc.job;
 
-import org.json.JSONObject;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 
-public interface IJSONOutputFunction {
-    public JSONObject invoke(String[] arguments) throws Exception;
+public interface IConnectorDescriptorVisitor {
+    public void visit(IConnectorDescriptor conn) throws HyracksException;
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/IJobStatusConditionVariable.java
similarity index 76%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
copy to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/IJobStatusConditionVariable.java
index 75edf2b..57ee9b0 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/IJobStatusConditionVariable.java
@@ -12,10 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.web.handlers.util;
+package edu.uci.ics.hyracks.control.cc.job;
 
-import org.json.JSONObject;
-
-public interface IJSONOutputFunction {
-    public JSONObject invoke(String[] arguments) throws Exception;
+public interface IJobStatusConditionVariable {
+    public void waitForCompletion() throws Exception;
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/IOperatorDescriptorVisitor.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/IOperatorDescriptorVisitor.java
index 65d8771..840faef 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/IOperatorDescriptorVisitor.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/IOperatorDescriptorVisitor.java
@@ -15,7 +15,8 @@
 package edu.uci.ics.hyracks.control.cc.job;
 
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 
 public interface IOperatorDescriptorVisitor {
-    public void visit(IOperatorDescriptor op) throws Exception;
+    public void visit(IOperatorDescriptor op) throws HyracksException;
 }
\ 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
new file mode 100644
index 0000000..284c92f
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobAttempt.java
@@ -0,0 +1,130 @@
+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/JobPlanBuilder.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanBuilder.java
similarity index 84%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanBuilder.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanBuilder.java
index d5056d3..bafa7e5 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanBuilder.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanBuilder.java
@@ -1,4 +1,4 @@
-package edu.uci.ics.hyracks.control.cc;
+package edu.uci.ics.hyracks.control.cc.job;
 
 import java.util.ArrayList;
 import java.util.EnumSet;
@@ -6,6 +6,7 @@
 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;
 
@@ -30,22 +31,22 @@
     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);
+                    + task.getActivityNodeId() + ":" + taskInputIndex);
         }
         insertIntoIndexedMap(plan.getTaskInputMap(), task.getActivityNodeId(), taskInputIndex, operatorInputIndex);
-        insertIntoIndexedMap(plan.getOperatorInputMap(), task.getOwner().getOperatorId(), operatorInputIndex, task
-            .getActivityNodeId());
+        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);
+                    + task.getActivityNodeId() + ":" + taskOutputIndex);
         }
         insertIntoIndexedMap(plan.getTaskOutputMap(), task.getActivityNodeId(), taskOutputIndex, operatorOutputIndex);
-        insertIntoIndexedMap(plan.getOperatorOutputMap(), task.getOwner().getOperatorId(), operatorOutputIndex, task
-            .getActivityNodeId());
+        insertIntoIndexedMap(plan.getOperatorOutputMap(), task.getOwner().getOperatorId(), operatorOutputIndex,
+                task.getActivityNodeId());
     }
 
     @Override
@@ -70,8 +71,8 @@
         }
     }
 
-    public void init(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
-        plan = new JobPlan(jobSpec, jobFlags);
+    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) {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanner.java
similarity index 86%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanner.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanner.java
index af3b77f..900e5ee 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/JobPlanner.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanner.java
@@ -12,9 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc;
+package edu.uci.ics.hyracks.control.cc.job;
 
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -28,13 +27,9 @@
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobPlan;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.api.job.JobStage;
 import edu.uci.ics.hyracks.api.util.Pair;
-import edu.uci.ics.hyracks.control.cc.job.IOperatorDescriptorVisitor;
-import edu.uci.ics.hyracks.control.cc.job.PlanUtils;
 
 public class JobPlanner {
     private static final Logger LOGGER = Logger.getLogger(JobPlanner.class.getName());
@@ -73,7 +68,7 @@
         return null;
     }
 
-    private JobStage inferStages(JobPlan plan) throws Exception {
+    private JobStage inferStages(JobPlan plan) {
         JobSpecification spec = plan.getJobSpecification();
 
         /*
@@ -131,7 +126,7 @@
     }
 
     private void merge(Map<ActivityNodeId, JobStage> eqSetMap, Set<JobStage> eqSets, ActivityNodeId t1,
-        ActivityNodeId t2) {
+            ActivityNodeId t2) {
         JobStage stage1 = eqSetMap.get(t1);
         Set<ActivityNodeId> s1 = stage1.getTasks();
         JobStage stage2 = eqSetMap.get(t2);
@@ -151,19 +146,8 @@
         }
     }
 
-    public JobPlan plan(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
-        final JobPlanBuilder builder = new JobPlanBuilder();
-        builder.init(jobSpec, jobFlags);
-        PlanUtils.visit(jobSpec, new IOperatorDescriptorVisitor() {
-            @Override
-            public void visit(IOperatorDescriptor op) throws Exception {
-                op.contributeTaskGraph(builder);
-            }
-        });
-        JobPlan plan = builder.getPlan();
-        JobStage endStage = inferStages(plan);
-        plan.setEndStage(endStage);
-
-        return plan;
+    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
new file mode 100644
index 0000000..5766288
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.job;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.control.cc.scheduler.IScheduler;
+
+public class JobRun implements IJobStatusConditionVariable {
+    private final JobPlan plan;
+    private final List<JobAttempt> attempts;
+    private JobStatus status;
+    private Set<ConstraintExpression> constraints;
+
+    public JobRun(JobPlan plan, Set<ConstraintExpression> constraints) {
+        this.plan = plan;
+        attempts = new ArrayList<JobAttempt>();
+        this.constraints = constraints;
+    }
+
+    public JobPlan getJobPlan() {
+        return plan;
+    }
+
+    public synchronized void setStatus(JobStatus status) {
+        this.status = status;
+        notifyAll();
+    }
+
+    public 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;
+    }
+
+    @Override
+    public synchronized void waitForCompletion() throws Exception {
+        while (status != JobStatus.TERMINATED && status != JobStatus.FAILURE) {
+            wait();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobStage.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStage.java
similarity index 97%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobStage.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStage.java
index 41187f1..8e4013d 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobStage.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStage.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;
+package edu.uci.ics.hyracks.control.cc.job;
 
 import java.io.Serializable;
 import java.util.HashSet;
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
new file mode 100644
index 0000000..61db770
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStageAttempt.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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/PlanUtils.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/PlanUtils.java
index 274ffdb..754094a 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/PlanUtils.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/PlanUtils.java
@@ -17,12 +17,14 @@
 import java.util.HashSet;
 import java.util.Set;
 
+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.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 public class PlanUtils {
-    public static void visit(JobSpecification spec, IOperatorDescriptorVisitor visitor) throws Exception {
+    public static void visit(JobSpecification spec, IOperatorDescriptorVisitor visitor) throws HyracksException {
         Set<OperatorDescriptorId> seen = new HashSet<OperatorDescriptorId>();
         for (IOperatorDescriptor op : spec.getOperatorMap().values()) {
             visitOperator(visitor, seen, op);
@@ -30,10 +32,16 @@
     }
 
     private static void visitOperator(IOperatorDescriptorVisitor visitor, Set<OperatorDescriptorId> seen,
-            IOperatorDescriptor op) throws Exception {
+            IOperatorDescriptor op) throws HyracksException {
         if (!seen.contains(op)) {
             visitor.visit(op);
         }
         seen.add(op.getOperatorId());
     }
+
+    public static void visit(JobSpecification spec, IConnectorDescriptorVisitor visitor) throws HyracksException {
+        for (IConnectorDescriptor c : spec.getConnectorMap().values()) {
+            visitor.visit(c);
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/IJobManager.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManager.java
similarity index 96%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/IJobManager.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManager.java
index 88076c9..ef87c4f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/IJobManager.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManager.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc;
+package edu.uci.ics.hyracks.control.cc.job.manager;
 
 import java.util.EnumSet;
 import java.util.Map;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/IJobManagerQueryInterface.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManagerQueryInterface.java
similarity index 87%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/IJobManagerQueryInterface.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManagerQueryInterface.java
index 221fc10..fbdf941 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/IJobManagerQueryInterface.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManagerQueryInterface.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc;
+package edu.uci.ics.hyracks.control.cc.job.manager;
 
 import java.util.UUID;
 
@@ -26,5 +26,5 @@
 
     public JSONObject getJobPlan(UUID jobId) throws Exception;
 
-    public JSONObject getJobProfile(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
new file mode 100644
index 0000000..be42e22
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/JobLifecycleHelper.java
@@ -0,0 +1,27 @@
+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/ApplicationDestroyEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java
new file mode 100644
index 0000000..69c244a
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.job.manager.events;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+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 {
+    private final ClusterControllerService ccs;
+    private final String appName;
+    private FutureValue fv;
+
+    public ApplicationDestroyEvent(ClusterControllerService ccs, String appName, FutureValue fv) {
+        this.ccs = ccs;
+        this.appName = appName;
+        this.fv = fv;
+    }
+
+    @Override
+    public void run() {
+        final ApplicationContext appCtx = ccs.getApplicationMap().remove(appName);
+        if (appCtx == null) {
+            fv.setException(new HyracksException("No application with name: " + appName));
+            return;
+        }
+        List<RemoteOp<Void>> opList = new ArrayList<RemoteOp<Void>>();
+        for (final String nodeId : ccs.getNodeMap().keySet()) {
+            opList.add(new ApplicationDestroyer(nodeId, appName));
+        }
+        final RemoteOp[] ops = opList.toArray(new RemoteOp[opList.size()]);
+        ccs.getExecutor().execute(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    RemoteRunner.runRemote(ccs, ops, null);
+                } catch (Exception e) {
+                    fv.setException(e);
+                    return;
+                }
+                ccs.getJobQueue().schedule(new Runnable() {
+                    @Override
+                    public void run() {
+                        try {
+                            appCtx.deinitialize();
+                        } catch (Exception e) {
+                            fv.setException(e);
+                        }
+                        fv.setValue(null);
+                    }
+                });
+            }
+        });
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..9f4ad8f
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationStartEvent.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.job.manager.events;
+
+import java.util.ArrayList;
+import java.util.List;
+
+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.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 {
+    private final ClusterControllerService ccs;
+    private final String appName;
+    private final FutureValue fv;
+
+    public ApplicationStartEvent(ClusterControllerService ccs, String appName, FutureValue fv) {
+        this.ccs = ccs;
+        this.appName = appName;
+        this.fv = fv;
+    }
+
+    @Override
+    public void run() {
+        ApplicationContext appCtx = ccs.getApplicationMap().get(appName);
+        if (appCtx == null) {
+            fv.setException(new HyracksException("No application with name: " + appName));
+            return;
+        }
+        try {
+            appCtx.initializeClassPath();
+            appCtx.initialize();
+            final byte[] distributedState = JavaSerializationUtils.serialize(appCtx.getDestributedState());
+            final boolean deployHar = appCtx.containsHar();
+            List<RemoteOp<Void>> opList = new ArrayList<RemoteOp<Void>>();
+            for (final String nodeId : ccs.getNodeMap().keySet()) {
+                opList.add(new ApplicationStarter(nodeId, appName, deployHar, distributedState));
+            }
+            final RemoteOp[] ops = opList.toArray(new RemoteOp[opList.size()]);
+            ccs.getExecutor().execute(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        RemoteRunner.runRemote(ccs, ops, null);
+                        fv.setValue(null);
+                    } catch (Exception e) {
+                        fv.setException(e);
+                    }
+                }
+            });
+        } catch (Exception e) {
+            fv.setException(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/GetJobProfileJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java
new file mode 100644
index 0000000..dc397ef
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.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 java.util.UUID;
+
+import org.json.JSONObject;
+
+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;
+
+public class GetJobProfileJSONEvent extends SynchronizableRunnable {
+    private final ClusterControllerService ccs;
+    private final UUID jobId;
+    private final int attempt;
+    private JSONObject profile;
+
+    public GetJobProfileJSONEvent(ClusterControllerService ccs, UUID jobId, int attempt) {
+        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) {
+            return;
+        }
+        List<JobAttempt> attempts = run.getAttempts();
+        if (attempts.size() <= attempt) {
+            return;
+        }
+        JobAttempt ja = attempts.get(attempt);
+        profile = ja.getJobProfile().toJSON();
+    }
+
+    public JSONObject getProfile() {
+        return profile;
+    }
+}
\ 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
new file mode 100644
index 0000000..6248925
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSpecificationJSONEvent.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.job.manager.events;
+
+import java.util.UUID;
+
+import org.json.JSONObject;
+
+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;
+
+public class GetJobSpecificationJSONEvent extends SynchronizableRunnable {
+    private final ClusterControllerService ccs;
+    private final UUID jobId;
+    private JSONObject spec;
+
+    public GetJobSpecificationJSONEvent(ClusterControllerService ccs, UUID jobId) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        JobRun run = ccs.getRunMap().get(jobId);
+        if (run == null) {
+            spec = new JSONObject();
+            return;
+        }
+        spec = run.getJobPlan().getJobSpecification().toJSON();
+    }
+
+    public JSONObject getSpecification() {
+        return spec;
+    }
+}
\ 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
new file mode 100644
index 0000000..c80974a
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusConditionVariableEvent.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.job.manager.events;
+
+import java.util.UUID;
+
+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;
+
+public class GetJobStatusConditionVariableEvent extends SynchronizableRunnable {
+    private final ClusterControllerService ccs;
+    private final UUID jobId;
+    private IJobStatusConditionVariable cVar;
+
+    public GetJobStatusConditionVariableEvent(ClusterControllerService ccs, UUID jobId) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        cVar = ccs.getRunMap().get(jobId);
+    }
+
+    public IJobStatusConditionVariable getConditionVariable() {
+        return cVar;
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..9b15e5c
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusEvent.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 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.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+
+public class GetJobStatusEvent extends SynchronizableRunnable {
+    private final ClusterControllerService ccs;
+    private final UUID jobId;
+    private JobStatus status;
+
+    public GetJobStatusEvent(ClusterControllerService ccs, UUID jobId) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        JobRun run = ccs.getRunMap().get(jobId);
+        status = run == null ? null : run.getStatus();
+    }
+
+    public JobStatus getStatus() {
+        return status;
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..74851e5
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSummariesJSONEvent.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.job.manager.events;
+
+import org.json.JSONArray;
+import org.json.JSONObject;
+
+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;
+
+public class GetJobSummariesJSONEvent extends SynchronizableRunnable {
+    private final ClusterControllerService ccs;
+    private JSONArray summaries;
+
+    public GetJobSummariesJSONEvent(ClusterControllerService ccs) {
+        this.ccs = ccs;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        summaries = new JSONArray();
+        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("status", run.getStatus().toString());
+            summaries.put(jo);
+        }
+    }
+
+    public JSONArray getSummaries() {
+        return summaries;
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..7554e62
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeEvent.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.manager.events;
+
+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;
+
+public class GetNodeEvent extends SynchronizableRunnable {
+    private final ClusterControllerService ccs;
+    private final String nodeId;
+    private NodeControllerState state;
+
+    public GetNodeEvent(ClusterControllerService ccs, String nodeId) {
+        this.ccs = ccs;
+        this.nodeId = nodeId;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        state = ccs.getNodeMap().get(nodeId);
+    }
+
+    public NodeControllerState getNodeState() {
+        return state;
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..9a0eff1
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAbortEvent.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.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
new file mode 100644
index 0000000..7dc6496
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAttemptStartEvent.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 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
new file mode 100644
index 0000000..c4ed077
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCleanupEvent.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.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.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.remote.RemoteRunner;
+import edu.uci.ics.hyracks.control.cc.remote.ops.JobCompleteNotifier;
+
+public class JobCleanupEvent implements Runnable {
+    private ClusterControllerService ccs;
+    private UUID jobId;
+    private int attempt;
+    private JobStatus status;
+
+    public JobCleanupEvent(ClusterControllerService ccs, UUID jobId, int attempt, JobStatus status) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.attempt = attempt;
+        this.status = status;
+    }
+
+    @Override
+    public void run() {
+        final JobRun run = ccs.getRunMap().get(jobId);
+        final JobAttempt ja = run.getAttempts().get(attempt);
+        Set<String> targetNodes = ja.getParticipatingNodeIds();
+        final JobCompleteNotifier[] jcns = new JobCompleteNotifier[targetNodes.size()];
+        int i = 0;
+        for (String n : targetNodes) {
+            jcns[i++] = new JobCompleteNotifier(n, jobId);
+        }
+        ccs.getExecutor().execute(new Runnable() {
+            @Override
+            public void run() {
+                if (jcns.length > 0) {
+                    try {
+                        RemoteRunner.runRemote(ccs, jcns, null);
+                    } catch (Exception e) {
+                        e.printStackTrace();
+                    }
+                }
+                ccs.getJobQueue().schedule(new Runnable() {
+                    @Override
+                    public void run() {
+                        CCApplicationContext appCtx = ccs.getApplicationMap().get(ja.getPlan().getApplicationName());
+                        if (appCtx != null) {
+                            try {
+                                appCtx.notifyJobFinish(jobId);
+                            } catch (HyracksException e) {
+                                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);
+                    }
+                });
+            }
+        });
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..3b03d25
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCreateEvent.java
@@ -0,0 +1,110 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobPlan;
+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.JobRun;
+import edu.uci.ics.hyracks.control.cc.job.PlanUtils;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+
+public class JobCreateEvent extends SynchronizableRunnable {
+    private final ClusterControllerService ccs;
+    private final byte[] jobSpec;
+    private final EnumSet<JobFlag> jobFlags;
+    private final UUID jobId;
+    private final String appName;
+
+    public JobCreateEvent(ClusterControllerService ccs, UUID jobId, String appName, byte[] jobSpec,
+            EnumSet<JobFlag> jobFlags) {
+        this.jobId = jobId;
+        this.ccs = ccs;
+        this.jobSpec = jobSpec;
+        this.jobFlags = jobFlags;
+        this.appName = appName;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
+        if (appCtx == null) {
+            throw new HyracksException("No application with id " + appName + " found");
+        }
+        JobSpecification spec = appCtx.createJobSpecification(jobId, jobSpec);
+        JobRun run = plan(jobId, spec, jobFlags);
+        run.setStatus(JobStatus.INITIALIZED);
+
+        ccs.getRunMap().put(jobId, run);
+        appCtx.notifyJobCreation(jobId, spec);
+    }
+
+    public UUID getJobId() {
+        return jobId;
+    }
+
+    private JobRun plan(UUID jobId, JobSpecification jobSpec, 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);
+            }
+        });
+        PlanUtils.visit(jobSpec, new IConnectorDescriptorVisitor() {
+            @Override
+            public void visit(IConnectorDescriptor conn) {
+                conn.contributeSchedulingConstraints(acceptor, plan);
+            }
+        });
+        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
new file mode 100644
index 0000000..742885e
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.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.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.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+
+public class JobStartEvent extends SynchronizableRunnable {
+    private final ClusterControllerService ccs;
+    private final UUID jobId;
+
+    public JobStartEvent(ClusterControllerService ccs, UUID jobId) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        JobRun run = ccs.getRunMap().get(jobId);
+        if (run == null) {
+            throw new Exception("Unable to find job with id = " + jobId);
+        }
+        if (run.getStatus() != JobStatus.INITIALIZED) {
+            throw new Exception("Job already started");
+        }
+        run.setStatus(JobStatus.RUNNING);
+
+        new JobAttemptStartEvent(ccs, jobId).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/NodeHeartbeatEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.java
new file mode 100644
index 0000000..348d703
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.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.control.cc.job.manager.events;
+
+import java.util.Map;
+
+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;
+
+public class NodeHeartbeatEvent extends SynchronizableRunnable {
+    private final ClusterControllerService ccs;
+    private final String nodeId;
+
+    public NodeHeartbeatEvent(ClusterControllerService ccs, String nodeId) {
+        this.ccs = ccs;
+        this.nodeId = nodeId;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
+        NodeControllerState state = nodeMap.get(nodeId);
+        if (state != null) {
+            state.notifyHeartbeat();
+        }
+    }
+}
\ 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
new file mode 100644
index 0000000..0754dc1
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterNodeEvent.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 java.util.Map;
+
+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;
+
+public class RegisterNodeEvent extends SynchronizableRunnable {
+    private final ClusterControllerService ccs;
+    private final String nodeId;
+    private final NodeControllerState state;
+
+    public RegisterNodeEvent(ClusterControllerService ccs, String nodeId, NodeControllerState state) {
+        this.ccs = ccs;
+        this.nodeId = nodeId;
+        this.state = state;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
+        if (nodeMap.containsKey(nodeId)) {
+            throw new Exception("Node with this name already registered.");
+        }
+        nodeMap.put(nodeId, state);
+    }
+}
\ 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
new file mode 100644
index 0000000..38fb7ae
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.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.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.logging.Logger;
+
+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;
+
+public class RemoveDeadNodesEvent implements Runnable {
+    private static Logger LOGGER = Logger.getLogger(RemoveDeadNodesEvent.class.getName());
+
+    private final ClusterControllerService ccs;
+
+    public RemoveDeadNodesEvent(ClusterControllerService ccs) {
+        this.ccs = ccs;
+    }
+
+    @Override
+    public void run() {
+        Set<String> deadNodes = new HashSet<String>();
+        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
+        for (Map.Entry<String, NodeControllerState> e : nodeMap.entrySet()) {
+            NodeControllerState state = e.getValue();
+            if (state.incrementLastHeartbeatDuration() >= ccs.getConfig().maxHeartbeatLapsePeriods) {
+                deadNodes.add(e.getKey());
+                LOGGER.info(e.getKey() + " considered dead");
+            }
+        }
+        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));
+            }
+        }
+    }
+}
\ 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
new file mode 100644
index 0000000..21c16c8
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ReportProfilesEvent.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.cc.job.manager.events;
+
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.job.profiling.om.JobProfile;
+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 ReportProfilesEvent implements Runnable {
+    private final ClusterControllerService ccs;
+    private final List<JobProfile> profiles;
+
+    public ReportProfilesEvent(ClusterControllerService ccs, List<JobProfile> profiles) {
+        this.ccs = ccs;
+        this.profiles = profiles;
+    }
+
+    @Override
+    public void run() {
+        Map<UUID, 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);
+            }
+        }
+    }
+}
\ 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
new file mode 100644
index 0000000..ba1c711
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ScheduleRunnableStagesEvent.java
@@ -0,0 +1,169 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.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
new file mode 100644
index 0000000..8f07d09
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletCompleteEvent.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.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
new file mode 100644
index 0000000..34dbeb4
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletFailureEvent.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.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/UnregisterNodeEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.java
new file mode 100644
index 0000000..fc1ba03
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.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.control.cc.job.manager.events;
+
+import java.util.Map;
+
+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;
+
+public class UnregisterNodeEvent extends SynchronizableRunnable {
+    private final ClusterControllerService ccs;
+    private final String nodeId;
+
+    public UnregisterNodeEvent(ClusterControllerService ccs, String nodeId) {
+        this.ccs = ccs;
+        this.nodeId = nodeId;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
+        nodeMap.remove(nodeId);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/FutureValue.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/FutureValue.java
new file mode 100644
index 0000000..25378c6
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/FutureValue.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.jobqueue;
+
+public class FutureValue {
+    private boolean done;
+
+    private Object value;
+
+    private Exception e;
+
+    public FutureValue() {
+        done = false;
+        value = null;
+        e = null;
+    }
+
+    public synchronized void setValue(Object value) {
+        done = true;
+        this.value = value;
+        e = null;
+        notifyAll();
+    }
+
+    public synchronized void setException(Exception e) {
+        done = true;
+        this.e = e;
+        value = null;
+        notifyAll();
+    }
+
+    public synchronized void reset() {
+        done = false;
+        value = null;
+        e = null;
+        notifyAll();
+    }
+
+    public synchronized Object get() throws Exception {
+        while (!done) {
+            wait();
+        }
+        if (e != null) {
+            throw e;
+        }
+        return value;
+    }
+}
\ 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
new file mode 100644
index 0000000..f533ad0
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/JobQueue.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.cc.jobqueue;
+
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Logger;
+
+public class JobQueue {
+    private static final Logger LOGGER = Logger.getLogger(JobQueue.class.getName());
+
+    private final LinkedBlockingQueue<Runnable> queue;
+    private final JobThread thread;
+
+    public JobQueue() {
+        queue = new LinkedBlockingQueue<Runnable>();
+        thread = new JobThread();
+        thread.start();
+    }
+
+    public void schedule(Runnable runnable) {
+        LOGGER.info("Scheduling: " + runnable);
+        queue.offer(runnable);
+    }
+
+    public void scheduleAndSync(SynchronizableRunnable sRunnable) throws Exception {
+        schedule(sRunnable);
+        sRunnable.sync();
+    }
+
+    private class JobThread extends Thread {
+        JobThread() {
+            setDaemon(true);
+        }
+
+        @Override
+        public void run() {
+            Runnable r;
+            while (true) {
+                try {
+                    r = queue.take();
+                } catch (InterruptedException e) {
+                    continue;
+                }
+                try {
+                    r.run();
+                } 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/jobqueue/SynchronizableRunnable.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableRunnable.java
new file mode 100644
index 0000000..53315b7
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableRunnable.java
@@ -0,0 +1,41 @@
+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/web/handlers/util/IJSONOutputFunction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
similarity index 76%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
copy to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
index 75edf2b..c728b0b 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
@@ -12,10 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.web.handlers.util;
+package edu.uci.ics.hyracks.control.cc.remote;
 
-import org.json.JSONObject;
+public interface Accumulator<T, R> {
+    public void accumulate(T o);
 
-public interface IJSONOutputFunction {
-    public JSONObject invoke(String[] arguments) throws Exception;
+    public R getResult();
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
similarity index 73%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
copy to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
index 75edf2b..c0a35cb 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
@@ -12,10 +12,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.web.handlers.util;
+package edu.uci.ics.hyracks.control.cc.remote;
 
-import org.json.JSONObject;
+import edu.uci.ics.hyracks.api.control.INodeController;
 
-public interface IJSONOutputFunction {
-    public JSONObject invoke(String[] arguments) throws Exception;
+public interface RemoteOp<T> {
+    public String getNodeId();
+
+    public T execute(INodeController node) throws Exception;
 }
\ No newline at end of file
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
new file mode 100644
index 0000000..9bd4375
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.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.cc.remote;
+
+import java.util.List;
+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;
+
+public class RemoteRunner {
+    public static <T, R> R runRemote(ClusterControllerService ccs, final RemoteOp<T>[] remoteOps,
+            final Accumulator<T, R> accumulator) throws Exception {
+        final Semaphore installComplete = new Semaphore(remoteOps.length);
+        final List<Exception> errors = new Vector<Exception>();
+        for (final RemoteOp<T> remoteOp : remoteOps) {
+            NodeControllerState nodeState = ccs.getNodeMap().get(remoteOp.getNodeId());
+            final INodeController node = nodeState.getNodeController();
+
+            installComplete.acquire();
+            Runnable remoteRunner = new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        T t = remoteOp.execute(node);
+                        if (accumulator != null) {
+                            synchronized (accumulator) {
+                                accumulator.accumulate(t);
+                            }
+                        }
+                    } catch (Exception e) {
+                        errors.add(e);
+                    } finally {
+                        installComplete.release();
+                    }
+                }
+            };
+
+            ccs.getExecutor().execute(remoteRunner);
+        }
+        installComplete.acquire(remoteOps.length);
+        if (!errors.isEmpty()) {
+            throw errors.get(0);
+        }
+        return accumulator == null ? null : accumulator.getResult();
+    }
+}
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
new file mode 100644
index 0000000..f8ca6af
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.remote.ops;
+
+import edu.uci.ics.hyracks.api.control.INodeController;
+import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
+
+public class ApplicationDestroyer implements RemoteOp<Void> {
+    private String nodeId;
+    private String appName;
+
+    public ApplicationDestroyer(String nodeId, String appName) {
+        this.nodeId = nodeId;
+        this.appName = appName;
+    }
+
+    @Override
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    @Override
+    public Void execute(INodeController node) throws Exception {
+        node.destroyApplication(appName);
+        return null;
+    }
+
+    @Override
+    public String toString() {
+        return "Destroyed application: " + appName;
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..af61249
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.remote.ops;
+
+import edu.uci.ics.hyracks.api.control.INodeController;
+import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
+
+public class ApplicationStarter implements RemoteOp<Void> {
+    private String nodeId;
+    private String appName;
+    private boolean deployHar;
+    private byte[] distributedState;
+
+    public ApplicationStarter(String nodeId, String appName, boolean deployHar, byte[] distributedState) {
+        this.nodeId = nodeId;
+        this.appName = appName;
+        this.deployHar = deployHar;
+        this.distributedState = distributedState;
+    }
+
+    @Override
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    @Override
+    public Void execute(INodeController node) throws Exception {
+        node.createApplication(appName, deployHar, distributedState);
+        return null;
+    }
+
+    @Override
+    public String toString() {
+        return "Started application: " + appName;
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..81a8610
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.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.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 JobCompleteNotifier implements RemoteOp<Void> {
+    private String nodeId;
+    private UUID jobId;
+
+    public JobCompleteNotifier(String nodeId, UUID jobId) {
+        this.nodeId = nodeId;
+        this.jobId = jobId;
+    }
+
+    @Override
+    public Void execute(INodeController node) throws Exception {
+        node.cleanUpJob(jobId);
+        return null;
+    }
+
+    @Override
+    public String toString() {
+        return jobId + " Cleaning Up";
+    }
+
+    @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/JobletAborter.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobletAborter.java
new file mode 100644
index 0000000..b50aa8e
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobletAborter.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.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
new file mode 100644
index 0000000..2fe67ac
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase1Installer.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
new file mode 100644
index 0000000..99d7c9f
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase2Installer.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.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
new file mode 100644
index 0000000..02f9aa1
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase3Installer.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.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
new file mode 100644
index 0000000..8cffd08
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/PortMapMergingAccumulator.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.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
new file mode 100644
index 0000000..d62254b
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/StageStarter.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.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/web/handlers/util/IJSONOutputFunction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobAttemptSchedulerState.java
similarity index 76%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
copy to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobAttemptSchedulerState.java
index 75edf2b..f8fcb89 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobAttemptSchedulerState.java
@@ -12,10 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.web.handlers.util;
+package edu.uci.ics.hyracks.control.cc.scheduler;
 
-import org.json.JSONObject;
-
-public interface IJSONOutputFunction {
-    public JSONObject invoke(String[] arguments) throws Exception;
+public interface IJobAttemptSchedulerState {
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
similarity index 76%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
copy to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
index 75edf2b..2906ba6 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
@@ -12,10 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.web.handlers.util;
+package edu.uci.ics.hyracks.control.cc.scheduler;
 
-import org.json.JSONObject;
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
 
-public interface IJSONOutputFunction {
-    public JSONObject invoke(String[] arguments) throws Exception;
+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
new file mode 100644
index 0000000..3893a2d
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IScheduler.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.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/naive/NaiveScheduler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/naive/NaiveScheduler.java
new file mode 100644
index 0000000..f40e8cd
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/naive/NaiveScheduler.java
@@ -0,0 +1,212 @@
+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);
+                System.err.println("Constraints: " + opConstraints);
+                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) {
+        System.err.println(ce);
+        if (ce.getTag() == ExpressionTag.RELATIONAL) {
+            RelationalExpression re = (RelationalExpression) ce;
+            if (re.getOperator() == RelationalExpression.Operator.EQUAL) {
+                System.err.println("Left: " + re.getLeft());
+                System.err.println("Right: " + re.getRight());
+                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);
+            }
+        }
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..1a26e60
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/AdminConsoleHandler.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.web;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Map;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.eclipse.jetty.server.Request;
+import org.eclipse.jetty.server.handler.AbstractHandler;
+
+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;
+
+public class AdminConsoleHandler extends AbstractHandler {
+    private ClusterControllerService ccs;
+
+    public AdminConsoleHandler(ClusterControllerService ccs) {
+        this.ccs = ccs;
+    }
+
+    @Override
+    public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response)
+            throws IOException, ServletException {
+        if (!"/".equals(target)) {
+            return;
+        }
+        response.setContentType("text/html;charset=utf-8");
+        response.setStatus(HttpServletResponse.SC_OK);
+        baseRequest.setHandled(true);
+        final PrintWriter writer = response.getWriter();
+        writer.println("<html><head><title>Hyracks Admin Console</title></head><body>");
+        writer.println("<h1>Hyracks Admin Console</h1>");
+        writer.println("<h2>Node Controllers</h2>");
+        writer.println("<table><tr><td>Node Id</td><td>Host</td></tr>");
+        try {
+            ccs.getJobQueue().scheduleAndSync(new SynchronizableRunnable() {
+                @Override
+                protected void doRun() throws Exception {
+                    for (Map.Entry<String, NodeControllerState> e : ccs.getNodeMap().entrySet()) {
+                        try {
+                            writer.print("<tr><td>");
+                            writer.print(e.getKey());
+                            writer.print("</td><td>");
+                            writer.print(e.getValue().getLastHeartbeatDuration());
+                            writer.print("</td></tr>");
+                        } catch (Exception ex) {
+                        }
+                    }
+                }
+            });
+        } catch (Exception e) {
+            throw new IOException(e);
+        }
+        writer.println("</table>");
+        writer.println("</body></html>");
+        writer.flush();
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..65c156c
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.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.cc.web;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.io.IOUtils;
+import org.eclipse.jetty.http.HttpMethods;
+import org.eclipse.jetty.server.Request;
+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.common.application.ApplicationContext;
+
+public class ApplicationInstallationHandler extends AbstractHandler {
+    private ClusterControllerService ccs;
+
+    public ApplicationInstallationHandler(ClusterControllerService ccs) {
+        this.ccs = ccs;
+    }
+
+    @Override
+    public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response)
+            throws IOException, ServletException {
+        try {
+            while (target.startsWith("/")) {
+                target = target.substring(1);
+            }
+            while (target.endsWith("/")) {
+                target = target.substring(0, target.length() - 1);
+            }
+            String[] parts = target.split("/");
+            if (parts.length != 1) {
+                return;
+            }
+            final String appName = parts[0];
+            if (HttpMethods.PUT.equals(request.getMethod())) {
+                class OutputStreamGetter extends SynchronizableRunnable {
+                    private OutputStream os;
+
+                    @Override
+                    protected void doRun() throws Exception {
+                        ApplicationContext appCtx;
+                        appCtx = ccs.getApplicationMap().get(appName);
+                        if (appCtx != null) {
+                            os = appCtx.getHarOutputStream();
+                        }
+                    }
+                }
+                OutputStreamGetter r = new OutputStreamGetter();
+                try {
+                    ccs.getJobQueue().scheduleAndSync(r);
+                } catch (Exception e) {
+                    throw new IOException(e);
+                }
+                try {
+                    IOUtils.copyLarge(request.getInputStream(), r.os);
+                } finally {
+                    r.os.close();
+                }
+            } else if (HttpMethods.GET.equals(request.getMethod())) {
+                class InputStreamGetter extends SynchronizableRunnable {
+                    private InputStream is;
+
+                    @Override
+                    protected void doRun() throws Exception {
+                        ApplicationContext appCtx;
+                        appCtx = ccs.getApplicationMap().get(appName);
+                        if (appCtx != null && appCtx.containsHar()) {
+                            is = appCtx.getHarInputStream();
+                        }
+                    }
+                }
+                InputStreamGetter r = new InputStreamGetter();
+                try {
+                    ccs.getJobQueue().scheduleAndSync(r);
+                } catch (Exception e) {
+                    throw new IOException(e);
+                }
+                if (r.is == null) {
+                    response.setStatus(HttpServletResponse.SC_NOT_FOUND);
+                } else {
+                    response.setContentType("application/octet-stream");
+                    response.setStatus(HttpServletResponse.SC_OK);
+                    try {
+                        IOUtils.copyLarge(r.is, response.getOutputStream());
+                    } finally {
+                        r.is.close();
+                    }
+                }
+            }
+            baseRequest.setHandled(true);
+        } catch (IOException e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..716dfcb
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/RESTAPIFunction.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.web;
+
+import java.util.UUID;
+
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+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;
+import edu.uci.ics.hyracks.control.cc.web.util.IJSONOutputFunction;
+
+public class RESTAPIFunction implements IJSONOutputFunction {
+    private ClusterControllerService ccs;
+
+    public RESTAPIFunction(ClusterControllerService ccs) {
+        this.ccs = ccs;
+    }
+
+    @Override
+    public JSONObject invoke(String[] arguments) throws Exception {
+        JSONObject result = new JSONObject();
+        switch (arguments.length) {
+            case 1:
+                if (!"".equals(arguments[0])) {
+                    break;
+                }
+            case 0: {
+                GetJobSummariesJSONEvent gjse = new GetJobSummariesJSONEvent(ccs);
+                ccs.getJobQueue().scheduleAndSync(gjse);
+                result.put("result", gjse.getSummaries());
+                break;
+            }
+
+            case 2: {
+                UUID jobId = UUID.fromString(arguments[0]);
+
+                if ("spec".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);
+                    ccs.getJobQueue().scheduleAndSync(gjpe);
+                    result.put("result", gjpe.getProfile());
+                }
+                break;
+            }
+        }
+        return result;
+    }
+}
\ No newline at end of file
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 12dfe21..806a14e 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
@@ -17,16 +17,23 @@
 import org.eclipse.jetty.server.Connector;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.handler.ContextHandler;
 import org.eclipse.jetty.server.handler.ContextHandlerCollection;
 import org.eclipse.jetty.server.handler.HandlerCollection;
 import org.eclipse.jetty.server.nio.SelectChannelConnector;
 
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.web.util.JSONOutputRequestHandler;
+import edu.uci.ics.hyracks.control.cc.web.util.RoutingHandler;
+
 public class WebServer {
+    private final ClusterControllerService ccs;
     private final Server server;
     private final SelectChannelConnector connector;
     private final HandlerCollection handlerCollection;
 
-    public WebServer() throws Exception {
+    public WebServer(ClusterControllerService ccs) throws Exception {
+        this.ccs = ccs;
         server = new Server();
 
         connector = new SelectChannelConnector();
@@ -35,6 +42,23 @@
 
         handlerCollection = new ContextHandlerCollection();
         server.setHandler(handlerCollection);
+        addHandlers();
+    }
+
+    private void addHandlers() {
+        ContextHandler handler = new ContextHandler("/state");
+        RoutingHandler rh = new RoutingHandler();
+        rh.addHandler("jobs", new JSONOutputRequestHandler(new RESTAPIFunction(ccs)));
+        handler.setHandler(rh);
+        addHandler(handler);
+
+        handler = new ContextHandler("/admin");
+        handler.setHandler(new AdminConsoleHandler(ccs));
+        addHandler(handler);
+
+        handler = new ContextHandler("/applications");
+        handler.setHandler(new ApplicationInstallationHandler(ccs));
+        addHandler(handler);
     }
 
     public void setPort(int port) {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/util/IJSONOutputFunction.java
similarity index 92%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/util/IJSONOutputFunction.java
index 75edf2b..f3582fb 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/IJSONOutputFunction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/util/IJSONOutputFunction.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.web.handlers.util;
+package edu.uci.ics.hyracks.control.cc.web.util;
 
 import org.json.JSONObject;
 
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/JSONOutputRequestHandler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/util/JSONOutputRequestHandler.java
similarity index 96%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/JSONOutputRequestHandler.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/util/JSONOutputRequestHandler.java
index c6b8d9c..1cc4dcf 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/JSONOutputRequestHandler.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/util/JSONOutputRequestHandler.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.web.handlers.util;
+package edu.uci.ics.hyracks.control.cc.web.util;
 
 import java.io.IOException;
 
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/RoutingHandler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/util/RoutingHandler.java
similarity index 97%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/RoutingHandler.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/util/RoutingHandler.java
index 6afaf2a..21b5143 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/handlers/util/RoutingHandler.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/util/RoutingHandler.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.web.handlers.util;
+package edu.uci.ics.hyracks.control.cc.web.util;
 
 import java.io.IOException;
 import java.util.HashMap;
diff --git a/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/scheduler.olg b/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/scheduler.olg
deleted file mode 100644
index 8174098..0000000
--- a/hyracks-control-cc/src/main/resources/edu/uci/ics/hyracks/control/cc/scheduler.olg
+++ /dev/null
@@ -1,337 +0,0 @@
-program hyrackscc;
-
-import java.util.UUID;
-import java.util.Set;
-import java.util.Map;
-import java.util.HashMap;
-
-import jol.types.basic.Tuple;
-import jol.types.basic.TupleSet;
-
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-
-watch(job, a);
-watch(job, i);
-watch(job, d);
-
-define(activitystage_temp, keys(), {UUID, OperatorDescriptorId, ActivityNodeId, Integer});
-
-activitystage_INITIAL activitystage_temp(JobId, OperatorId, ActivityId, 0) :-
-    activitynode(JobId, OperatorId, ActivityId, _);
-
-activitystage_BLOCKED activitystage_temp(JobId, OperatorId2, ActivityId2, StageNumber) :-
-/* use activitystage_temp#insert() to trigger only on insert */
-    activitystage_temp(JobId, OperatorId1, ActivityId1, StageNumber1),
-    activitystage_temp(JobId, OperatorId2, ActivityId2, StageNumber2),
-    activityblocked(JobId, OperatorId1, ActivityId1, OperatorId2, ActivityId2),
-    StageNumber2 <= StageNumber1
-    {
-        StageNumber := StageNumber1 + 1;
-    };
-
-activitystage_PIPELINED_1 activitystage_temp(JobId, OperatorId2, ActivityId2, StageNumber) :-
-    activitystage_temp(JobId, OperatorId1, ActivityId1, StageNumber1),
-    activitystage_temp(JobId, OperatorId2, ActivityId2, StageNumber2),
-    activityconnection(JobId, OperatorId1, Operator1Port, edu.uci.ics.hyracks.api.dataflow.Direction.OUTPUT, ActivityId1, _),
-    activityconnection(JobId, OperatorId2, Operator2Port, edu.uci.ics.hyracks.api.dataflow.Direction.INPUT, ActivityId2, _),
-    connectordescriptor(JobId, _, OperatorId1, Operator1Port, OperatorId2, Operator2Port, _),
-    StageNumber1 != StageNumber2
-    {
-        StageNumber := java.lang.Math.max(StageNumber1, StageNumber2);
-    };
-
-activitystage_PIPELINED_2 activitystage_temp(JobId, OperatorId1, ActivityId1, StageNumber) :-
-    activitystage_temp(JobId, OperatorId1, ActivityId1, StageNumber1),
-    activitystage_temp(JobId, OperatorId2, ActivityId2, StageNumber2),
-    activityconnection(JobId, OperatorId1, Operator1Port, edu.uci.ics.hyracks.api.dataflow.Direction.OUTPUT, ActivityId1, _),
-    activityconnection(JobId, OperatorId2, Operator2Port, edu.uci.ics.hyracks.api.dataflow.Direction.INPUT, ActivityId2, _),
-    connectordescriptor(JobId, _, OperatorId1, Operator1Port, OperatorId2, Operator2Port, _),
-    StageNumber1 != StageNumber2
-    {
-        StageNumber := java.lang.Math.max(StageNumber1, StageNumber2);
-    };
-
-watch(activitystage_temp, a);
-
-watch(activityconnection, a);
-watch(activityblocked, a);
-watch(operatordescriptor, a);
-watch(connectordescriptor, a);
-
-watch(activitystage, a);
-watch(activitystage, i);
-watch(activitystage, d);
-
-define(activitystage, keys(0, 1, 2), {UUID, OperatorDescriptorId, ActivityNodeId, Integer});
-
-activitystage(JobId, OperatorId, ActivityId, max<StageNumber>) :-
-    activitystage_temp(JobId, OperatorId, ActivityId, StageNumber);
-
-define(jobstage, keys(0, 1), {UUID, Integer, UUID});
-
-jobstage(JobId, StageNumber, StageId) :-
-    activitystage(JobId, _, _, StageNumber)
-    {
-        StageId := java.util.UUID.randomUUID();
-    };
-
-watch(jobstage, a);
-
-define(jobattempt, keys(), {UUID, Integer});
-
-jobattempt(JobId, 0) :-
-    job(JobId, _, edu.uci.ics.hyracks.api.job.JobStatus.INITIALIZED, _, _, _),
-    jobstart(JobId, _);
-
-jobattempt(JobId, NextAttempt) :-
-    jobattempt(JobId, Attempt),
-    stagestart(JobId, _, Attempt),
-    abortcomplete(JobId, _, Attempt)
-    {
-        NextAttempt := Attempt + 1;
-    };
-
-job(JobId, AppName, Status, JobSpec, JobPlan, Stats) :-
-    job(JobId, AppName, Status, JobSpec, JobPlan, Stats),
-    profileupdate(JobId, _, JobletStats)
-    {
-        Stats.putAll(JobletStats);
-    };
-
-define(stagestart, keys(), {UUID, Integer, Integer});
-define(stagefinish, keys(0, 1, 2), {UUID, Integer, Integer, Map});
-
-watch(jobstart, i);
-
-stagestart_INITIAL stagestart(JobId, 0, Attempt) :-
-    jobattempt#insert(JobId, Attempt);
-
-update_job_status_RUNNING job(JobId, AppName, edu.uci.ics.hyracks.api.job.JobStatus.RUNNING, JobSpec, JobPlan, Stats) :-
-    job(JobId, AppName, edu.uci.ics.hyracks.api.job.JobStatus.INITIALIZED, JobSpec, JobPlan, Stats),
-    jobstart(JobId, _);
-
-stagestart_NEXT stagestart(JobId, NextStageNumber, Attempt) :-
-    stagestart(JobId, StageNumber, Attempt),
-    stagefinish#insert(StageId, StageNumber, Attempt, _)
-    {
-        NextStageNumber := StageNumber + 1;
-    };
-
-watch(stagestart, a);
-watch(stagestart, d);
-
-define(operatorlocationcandidates, keys(), {UUID, OperatorDescriptorId, String, Integer, Integer});
-
-operatorlocationcandidates(JobId, OperatorId, NodeId, Partition, Benefit) :-
-    operatorlocation(JobId, OperatorId, NodeId, Partition, Benefit),
-    availablenodes(NodeId);
-
-watch(availablenodes, a);
-watch(availablenodes, i);
-watch(availablenodes, d);
-
-define(availablenodecount, keys(0), {Integer, Integer});
-
-watch(availablenodecount, a);
-watch(availablenodecount, i);
-watch(availablenodecount, d);
-
-/* Try not using a 0 key -- Tyson feels that should work! */
-availablenodecount(0, count<NodeId>) :-
-    availablenodes(NodeId);
-
-watch(rankedavailablenodes, a);
-watch(rankedavailablenodes, i);
-watch(rankedavailablenodes, d);
-
-watch(operatorlocationcandidates, a);
-watch(operatorlocationcandidates, i);
-watch(operatorlocationcandidates, d);
-
-define(maxoperatorlocationbenefit, keys(0, 1, 2), {UUID, OperatorDescriptorId, Integer, Integer});
-
-maxoperatorlocationbenefit(JobId, OperatorId, Partition, max<Benefit>) :-
-    operatorlocationcandidates(JobId, OperatorId, _, Partition, Benefit);
-
-watch(maxoperatorlocationbenefit, a);
-watch(maxoperatorlocationbenefit, i);
-watch(maxoperatorlocationbenefit, d);
-
-define(attemptoperatorlocationdecision, keys(0, 1, 3, 4), {UUID, OperatorDescriptorId, String, Integer, Integer});
-
-watch(attemptoperatorlocationdecision, a);
-watch(attemptoperatorlocationdecision, i);
-watch(attemptoperatorlocationdecision, d);
-
-attemptoperatorlocationdecision(JobId, OperatorId, NodeId, Partition, Attempt) :-
-    jobattempt#insert(JobId, Attempt),
-    operatorlocationcandidates(JobId, OperatorId, NodeId, Partition, Benefit),
-    maxoperatorlocationbenefit(JobId, OperatorId, Partition, Benefit);
-
-attemptoperatorlocationdecision(JobId, OperatorId, NodeId, Partition, Attempt) :-
-    jobattempt#insert(JobId, Attempt),
-    operatorclonecountexpansiontotalorder(JobId, OperatorId, Partition, CloneRank),
-    rankedavailablenodes(NodeId, NodeRank),
-    availablenodecount(_, NodeCount),
-    NodeRank == CloneRank % NodeCount;
-
-define(operatorclonecount_temp, keys(), {UUID, OperatorDescriptorId, Integer, Integer});
-
-operatorclonecount_temp(JobId, OperatorId, NPartitions, 0) :-
-    operatorclonecount(JobId, OperatorId, NPartitions);
-
-define(operatorclonecountexpansiontotalorder, keys(0, 1, 2), {UUID, OperatorDescriptorId, Integer, Integer});
-
-operatorclonecountexpansiontotalorder(JobId, OperatorId, Partition, Rank) :-
-    expandpartitioncountconstraint(operatorclonecount_temp(JobId, OperatorId, Partition, Rank));
-
-watch(operatorclonecountexpansiontotalorder, a);
-watch(operatorclonecountexpansiontotalorder, i);
-watch(operatorclonecountexpansiontotalorder, d);
-
-watch(operatorclonecount, a);
-watch(operatorclonecount, i);
-watch(operatorclonecount, d);
-
-define(activitystart, keys(), {UUID, OperatorDescriptorId, ActivityNodeId, Integer, Integer, UUID, String, Integer});
-
-activitystart(JobId, OperatorId, ActivityId, StageNumber, Attempt, StageId, NodeId, Partition) :-
-    stagestart#insert(JobId, StageNumber, Attempt),
-    operatordescriptor(JobId, OperatorId, _, _),
-    activitystage(JobId, OperatorId, ActivityId, StageNumber),
-    jobstage(JobId, StageNumber, StageId),
-    attemptoperatorlocationdecision(JobId, OperatorId, NodeId, Partition, Attempt);
-
-watch(activitystart, a);
-
-define(stageletstart, keys(0, 1, 4, 5), {UUID, UUID, String, JobPlan, String, Integer, Set});
-
-stageletstart(JobId, StageId, AppName, JobPlan, NodeId, Attempt, set<ActivityInfo>) :-
-    activitystart#insert(JobId, _, ActivityId, StageNumber, Attempt, StageId, NodeId, Partition),
-    job(JobId, AppName, edu.uci.ics.hyracks.api.job.JobStatus.RUNNING, _, JobPlan, _)
-    {
-        ActivityInfo := [ActivityId, Partition];
-    };
-
-watch(stageletstart, a);
-watch(stageletstart, i);
-
-define(startmessage_agg, keys(0, 1, 2), {UUID, UUID, Integer, String, JobPlan, Set});
-
-startmessage_agg(JobId, StageId, Attempt, AppName, JobPlan, set<Tuple>) :-
-    stageletstart#insert(JobId, StageId, AppName, JobPlan, NodeId, Attempt, ActivityInfoSet),
-    availablenodes(NodeId),
-    ActivityInfoSet.size() != 0
-    {
-        Tuple := [NodeId, ActivityInfoSet];
-    };
-
-startmessage(JobId, StageId, Attempt, AppName, JobPlan, TSet) :-
-    startmessage_agg(JobId, StageId, Attempt, AppName, JobPlan, TSet);
-
-watch(startmessage, a);
-watch(startmessage, i);
-
-define(stageletabort, keys(0, 1, 3, 4), {UUID, UUID, JobPlan, String, Integer, Set});
-
-stageletabort(JobId, StageId, JobPlan, NodeId, Attempt, ActivityIdSet) :-
-    stageletfailure(JobId, StageId, NodeId, Attempt),
-    stageletstart(JobId, StageId, _, JobPlan, NodeId, Attempt, ActivityIdSet);
-
-stageletabort(JobId, StageId, JobPlan, NodeIdOther, Attempt, ActivityIdSet) :-
-    stageletstart(JobId, StageId, _, JobPlan, NodeId, Attempt, _),
-    stageletstart(JobId, StageId, _, _, NodeIdOther, Attempt, ActivityIdSet),
-    failednodes#insert(NodeId),
-    notin stageletcomplete(JobId, StageId, NodeId, Attempt, _);
-
-watch(stageletabort, a);
-watch(stageletabort, i);
-watch(stageletabort, d);
-
-define(stageabort, keys(0, 1, 2), {UUID, UUID, Integer, Set});
-
-stageabort(JobId, StageId, Attempt, set<NodeId>) :-
-    stageletabort#insert(JobId, StageId, _, NodeId, Attempt, _);
-
-define(abortmessage_agg, keys(0, 1, 2), {UUID, UUID, Integer, JobPlan, Set});
-
-abortmessage_agg(JobId, StageId, Attempt, JobPlan, set<Tuple>) :-
-    stageletabort#insert(JobId, StageId, JobPlan, NodeId, Attempt, ActivityIdSet),
-    availablenodes(NodeId)
-    {
-        Tuple := [NodeId, ActivityIdSet];
-    };
-
-abortmessage(JobId, StageId, Attempt, JobPlan, TSet) :-
-    abortmessage_agg(JobId, StageId, Attempt, JobPlan, TSet),
-    TSet.size() != 0;
-
-watch(abortmessage, a);
-watch(abortmessage, i);
-
-define(stageletabortcomplete, keys(), {UUID, UUID, String, Integer});
-
-stageletabortcomplete(JobId, StageId, NodeId, Attempt) :-
-    abortnotify(JobId, StageId, NodeId, Attempt);
-
-stageletabortcomplete(JobId, StageId, NodeId, Attempt) :-
-    stageletabort(JobId, StageId, _, NodeId, Attempt, _),
-    notin availablenodes(NodeId);
-
-define(stageletabortcomplete_agg, keys(0, 1, 2), {UUID, UUID, Integer, Set});
-
-stageletabortcomplete_agg(JobId, StageId, Attempt, set<NodeId>) :-
-    stageletabortcomplete(JobId, StageId, NodeId, Attempt);
-
-define(abortcomplete, keys(), {UUID, UUID, Integer});
-
-abortcomplete(JobId, StageId, Attempt) :-
-    stageletabortcomplete_agg(JobId, StageId, Attempt, NodeIdSet1),
-    stageabort(JobId, StageId, Attempt, NodeIdSet2),
-    NodeIdSet1.size() == NodeIdSet2.size();
-
-define(stageletcompletecount, keys(0, 1, 2), {UUID, UUID, Integer, Integer});
-
-stageletcompletecount(JobId, StageId, Attempt, count<NodeId>) :-
-    stageletcomplete(JobId, StageId, NodeId, Attempt, _);
-
-define(stageletcomplete_agg, keys(0, 1, 2), {UUID, UUID, Integer, Map});
-
-stageletcomplete_agg(JobId, StageId, Attempt, generic<(new HashMap()).putAll(Statistics)>) :-
-    stageletcomplete(JobId, StageId, NodeId, Attempt, Statistics);
-
-stagefinish(JobId, StageNumber, Attempt, SMap) :-
-    startmessage_agg(JobId, StageId, Attempt, _, _, TSet),
-    stageletcompletecount(JobId, StageId, Attempt, Count),
-    stageletcomplete_agg(JobId, StageId, Attempt, SMap),
-    jobstage(JobId, StageNumber, StageId),
-    TSet.size() == Count;
-
-update_job_status_TERMINATED job(JobId, AppName, edu.uci.ics.hyracks.api.job.JobStatus.TERMINATED, JobSpec, JobPlan, NewStats) :-
-    job(JobId, AppName, edu.uci.ics.hyracks.api.job.JobStatus.RUNNING, JobSpec, JobPlan, Stats),
-    stagestart#insert(JobId, StageNumber, Attempt),
-    stagefinish(JobId, _, Attempt, SMap),
-    notin jobstage(JobId, StageNumber)
-    {
-        NewStats := new HashMap();
-        NewStats.putAll(Stats);
-        NewStats.putAll(SMap);
-    };
-
-define(jobcleanup_agg, {UUID, String, Set});
-
-jobcleanup_agg(JobId, AppName, set<NodeId>) :-
-    stagestart#insert(JobId, StageNumber, Attempt),
-    stagefinish(JobId, _, Attempt, _),
-    attemptoperatorlocationdecision(JobId, _, NodeId, _, Attempt),
-    job(JobId, AppName, _, _, _, _),
-    notin jobstage(JobId, StageNumber);
-
-jobcleanup(JobId, AppName, NodeIdSet) :-
-    jobcleanup_agg(JobId, AppName, NodeIdSet);
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 d7b2e5d..6cc7450 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
@@ -16,9 +16,7 @@
 
 import java.nio.ByteBuffer;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
-import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.Executor;
 
@@ -31,14 +29,17 @@
 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.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.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.job.profiling.CounterContext;
 import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
 
-public class Joblet implements IHyracksJobletContext {
+public class Joblet implements IHyracksJobletContext, ICounterContext {
     private static final long serialVersionUID = 1L;
 
     private final NodeControllerService nodeController;
@@ -53,7 +54,7 @@
 
     private final Map<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>> envMap;
 
-    private final ICounterContext counterContext;
+    private final Map<String, Counter> counterMap;
 
     private final DefaultDeallocatableRegistry deallocatableRegistry;
 
@@ -66,7 +67,7 @@
         this.attempt = attempt;
         stageletMap = new HashMap<UUID, Stagelet>();
         envMap = new HashMap<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>>();
-        counterContext = new CounterContext(getJobId() + "." + nodeController.getId());
+        counterMap = new HashMap<String, Counter>();
         deallocatableRegistry = new DefaultDeallocatableRegistry();
         fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) appCtx.getRootContext().getIOManager());
     }
@@ -117,8 +118,7 @@
         return nodeController.getExecutor();
     }
 
-    public synchronized void notifyStageletComplete(UUID stageId, int attempt, Map<String, Long> stats)
-            throws Exception {
+    public synchronized void notifyStageletComplete(UUID stageId, int attempt, StageletProfile stats) throws Exception {
         stageletMap.remove(stageId);
         nodeController.notifyStageComplete(jobId, stageId, attempt, stats);
     }
@@ -132,19 +132,15 @@
         return nodeController;
     }
 
-    public void dumpProfile(Map<String, Long> counterDump) {
-        Set<UUID> stageIds;
-        synchronized (this) {
-            stageIds = new HashSet<UUID>(stageletMap.keySet());
+    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());
         }
-        for (UUID stageId : stageIds) {
-            Stagelet si;
-            synchronized (this) {
-                si = stageletMap.get(stageId);
-            }
-            if (si != null) {
-                si.dumpProfile(counterDump);
-            }
+        for (Stagelet si : stageletMap.values()) {
+            StageletProfile sProfile = new StageletProfile(si.getStageId());
+            si.dumpProfile(sProfile);
+            jProfile.getStageletProfiles().put(si.getStageId(), sProfile);
         }
     }
 
@@ -160,7 +156,7 @@
 
     @Override
     public ICounterContext getCounterContext() {
-        return counterContext;
+        return this;
     }
 
     @Override
@@ -191,4 +187,18 @@
     public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
         return fileFactory.createWorkspaceFile(prefix);
     }
+
+    public Map<UUID, Stagelet> getStageletMap() {
+        return stageletMap;
+    }
+
+    @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/NodeControllerService.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 19c8ed8..2702386 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
@@ -25,7 +25,6 @@
 import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Hashtable;
 import java.util.List;
 import java.util.Map;
@@ -78,6 +77,9 @@
 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.control.common.AbstractRemoteService;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
@@ -210,7 +212,7 @@
     @Override
     public Map<PortInstanceId, Endpoint> initializeJobletPhase1(String appName, UUID jobId, int attempt,
             byte[] planBytes, UUID stageId, Map<ActivityNodeId, Set<Integer>> tasks,
-            Map<OperatorDescriptorId, Set<Integer>> opPartitions) throws Exception {
+            Map<OperatorDescriptorId, Integer> opNumPartitions) throws Exception {
         try {
             LOGGER.log(Level.INFO, String.valueOf(jobId) + "[" + id + ":" + stageId + "]: Initializing Joblet Phase 1");
 
@@ -248,7 +250,7 @@
                 List<IConnectorDescriptor> inputs = plan.getTaskInputs(hanId);
                 for (int i : tasks.get(hanId)) {
                     IOperatorNodePushable hon = han.createPushRuntime(stagelet, joblet.getEnvironment(op, i), rdp, i,
-                            opPartitions.get(op.getOperatorId()).size());
+                            opNumPartitions.get(op.getOperatorId()));
                     OperatorRunnable or = new OperatorRunnable(stagelet, hon);
                     stagelet.setOperator(op.getOperatorId(), i, or);
                     if (inputs != null) {
@@ -272,8 +274,8 @@
                                 LOGGER.finest("Created endpoint " + piId + " -> " + endpoint);
                             }
                             portMap.put(piId, endpoint);
-                            IFrameReader reader = createReader(stagelet, conn, drlf, i, plan, stagelet, opPartitions
-                                    .get(producerOpId).size(), opPartitions.get(consumerOpId).size());
+                            IFrameReader reader = createReader(stagelet, conn, drlf, i, plan, stagelet,
+                                    opNumPartitions.get(producerOpId), opNumPartitions.get(consumerOpId));
                             or.setFrameReader(reader);
                         }
                     }
@@ -329,7 +331,7 @@
 
     @Override
     public void initializeJobletPhase2(String appName, UUID jobId, byte[] planBytes, UUID stageId,
-            Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Set<Integer>> opPartitions,
+            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");
@@ -374,8 +376,8 @@
                                 }
                             };
                             or.setFrameWriter(j, conn.createSendSideWriter(stagelet, plan.getJobSpecification()
-                                    .getConnectorRecordDescriptor(conn), edwFactory, i, opPartitions.get(producerOpId)
-                                    .size(), opPartitions.get(consumerOpId).size()), spec
+                                    .getConnectorRecordDescriptor(conn), edwFactory, i, opNumPartitions
+                                    .get(producerOpId), opNumPartitions.get(consumerOpId)), spec
                                     .getConnectorRecordDescriptor(conn));
                         }
                     }
@@ -444,7 +446,7 @@
     private synchronized Joblet getOrCreateLocalJoblet(UUID jobId, int attempt, INCApplicationContext appCtx)
             throws Exception {
         Joblet ji = jobletMap.get(jobId);
-        if (ji == null) {
+        if (ji == null || ji.getAttempt() != attempt) {
             ji = new Joblet(this, jobId, attempt, appCtx);
             jobletMap.put(jobId, ji);
         }
@@ -478,7 +480,7 @@
         }
     }
 
-    public void notifyStageComplete(UUID jobId, UUID stageId, int attempt, Map<String, Long> stats) throws Exception {
+    public void notifyStageComplete(UUID jobId, UUID stageId, int attempt, StageletProfile stats) throws Exception {
         try {
             ccs.notifyStageletComplete(jobId, stageId, attempt, id, stats);
         } catch (Exception e) {
@@ -533,24 +535,26 @@
         @Override
         public void run() {
             try {
-                Map<UUID, Map<String, Long>> counterDump = new HashMap<UUID, Map<String, Long>>();
-                Set<UUID> jobIds;
+                List<JobProfile> profiles;
                 synchronized (NodeControllerService.this) {
-                    jobIds = new HashSet<UUID>(jobletMap.keySet());
+                    profiles = new ArrayList<JobProfile>();
+                    for (Joblet ji : jobletMap.values()) {
+                        profiles.add(new JobProfile(ji.getJobId(), ji.getAttempt()));
+                    }
                 }
-                for (UUID jobId : jobIds) {
+                for (JobProfile jProfile : profiles) {
                     Joblet ji;
+                    JobletProfile jobletProfile = new JobletProfile(id);
                     synchronized (NodeControllerService.this) {
-                        ji = jobletMap.get(jobId);
+                        ji = jobletMap.get(jProfile.getJobId());
                     }
                     if (ji != null) {
-                        Map<String, Long> jobletCounterDump = new HashMap<String, Long>();
-                        ji.dumpProfile(jobletCounterDump);
-                        counterDump.put(jobId, jobletCounterDump);
+                        ji.dumpProfile(jobletProfile);
+                        jProfile.getJobletProfiles().put(id, jobletProfile);
                     }
                 }
-                if (!counterDump.isEmpty()) {
-                    cc.reportProfile(id, counterDump);
+                if (!profiles.isEmpty()) {
+                    cc.reportProfile(id, profiles);
                 }
             } catch (Exception e) {
                 e.printStackTrace();
@@ -559,14 +563,19 @@
     }
 
     @Override
-    public synchronized void abortJoblet(UUID jobId, UUID stageId) throws Exception {
+    public synchronized void abortJoblet(UUID jobId, int attempt) throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Aborting Job: " + jobId + ":" + attempt);
+        }
         Joblet ji = jobletMap.get(jobId);
         if (ji != null) {
-            Stagelet stagelet = ji.getStagelet(stageId);
-            if (stagelet != null) {
+            if (ji.getAttempt() == attempt) {
+                jobletMap.remove(jobId);
+            }
+            for (Stagelet stagelet : ji.getStageletMap().values()) {
                 stagelet.abort();
                 stagelet.close();
-                connectionManager.abortConnections(jobId, stageId);
+                connectionManager.abortConnections(jobId, stagelet.getStageId());
             }
         }
     }
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
index 3565445..eb283c2 100644
--- 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
@@ -21,7 +21,6 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
 import java.util.UUID;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -35,15 +34,17 @@
 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.job.profiling.CounterContext;
 import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
 import edu.uci.ics.hyracks.control.nc.runtime.OperatorRunnable;
 
-public class Stagelet implements IHyracksStageletContext {
+public class Stagelet implements IHyracksStageletContext, ICounterContext {
     private static final long serialVersionUID = 1L;
 
     private static final Logger LOGGER = Logger.getLogger(Stagelet.class.getName());
@@ -56,7 +57,7 @@
 
     private final Map<OperatorInstanceId, OperatorRunnable> honMap;
 
-    private final CounterContext stageletCounterContext;
+    private final Map<String, Counter> counterMap;
 
     private final IWorkspaceFileFactory fileFactory;
 
@@ -77,7 +78,7 @@
         pendingOperators = new HashSet<OperatorInstanceId>();
         started = false;
         honMap = new HashMap<OperatorInstanceId, OperatorRunnable>();
-        stageletCounterContext = new CounterContext(joblet.getJobId() + "." + stageId + "." + nodeId);
+        counterMap = new HashMap<String, Counter>();
         deallocatableRegistry = new DefaultDeallocatableRegistry();
         fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) joblet.getIOManager());
     }
@@ -90,10 +91,6 @@
         return honMap;
     }
 
-    public CounterContext getStageletCounterContext() {
-        return stageletCounterContext;
-    }
-
     public void setEndpointList(List<Endpoint> endpointList) {
         this.endpointList = endpointList;
     }
@@ -143,12 +140,7 @@
                     LOGGER.log(Level.INFO, joblet.getJobId() + ":" + stageId + ":" + opIId.getOperatorId() + ":"
                             + opIId.getPartition() + "(" + hon + ")" + ": ABORTED");
                     e.printStackTrace();
-                    // DO NOT UNCOMMENT THE FOLLOWING LINE.
-                    // The failure of an operator triggers a re-attempt of the job at the CC. If the failure was non-transient,
-                    // this will lead to an infinite number of attempts since there is no upper bount yet on how many times
-                    // a job is retried.
-
-                    // notifyOperatorFailure(opIId);
+                    notifyOperatorFailure(opIId);
                 }
             }
         });
@@ -158,10 +150,10 @@
         pendingOperators.remove(opIId);
         if (pendingOperators.isEmpty()) {
             try {
-                Map<String, Long> stats = new TreeMap<String, Long>();
-                dumpProfile(stats);
+                StageletProfile sProfile = new StageletProfile(stageId);
+                dumpProfile(sProfile);
                 close();
-                joblet.notifyStageletComplete(stageId, attempt, stats);
+                joblet.notifyStageletComplete(stageId, attempt, sProfile);
             } catch (Exception e) {
                 e.printStackTrace();
             }
@@ -183,8 +175,11 @@
         }
     }
 
-    public void dumpProfile(Map<String, Long> counterDump) {
-        stageletCounterContext.dump(counterDump);
+    public synchronized void dumpProfile(StageletProfile sProfile) {
+        Map<String, Long> dumpMap = sProfile.getCounters();
+        for (Counter c : counterMap.values()) {
+            dumpMap.put(c.getName(), c.get());
+        }
     }
 
     @Override
@@ -199,7 +194,7 @@
 
     @Override
     public ICounterContext getCounterContext() {
-        return stageletCounterContext;
+        return this;
     }
 
     @Override
@@ -230,4 +225,14 @@
     public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
         return fileFactory.createWorkspaceFile(prefix);
     }
+
+    @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;
+    }
 }
\ No newline at end of file
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 4f0ae2f..915d09b 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
@@ -31,7 +31,7 @@
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.ReflectionUtils;
 
-import edu.uci.ics.hyracks.api.constraints.PartitionCountConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -74,7 +74,7 @@
         }
         recordDescriptors[0] = DatatypeHelper.createKeyValueRecordDescriptor((Class<? extends Writable>) recordReader
                 .createKey().getClass(), (Class<? extends Writable>) recordReader.createValue().getClass());
-        this.setPartitionConstraint(new PartitionCountConstraint(splits.length));
+        PartitionConstraintHelper.addPartitionCountConstraint(spec, this, splits.length);
         inputSplitsProxy = new InputSplitsProxy(jobConf, splits);
         this.inputFormatClassName = inputFormat.getClass().getName();
     }
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 aaae486..0eebd93 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
@@ -19,8 +19,10 @@
 import org.json.JSONException;
 import org.json.JSONObject;
 
+import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
 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.JobSpecification;
 
 public abstract class AbstractConnectorDescriptor implements IConnectorDescriptor {
@@ -46,4 +48,9 @@
 
         return jconn;
     }
+
+    @Override
+    public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan) {
+        // do nothing
+    }
 }
\ 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 b8af305..7e97975 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
@@ -19,10 +19,11 @@
 import org.json.JSONException;
 import org.json.JSONObject;
 
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
 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.JobSpecification;
 
 public abstract class AbstractOperatorDescriptor implements IOperatorDescriptor {
@@ -30,8 +31,6 @@
 
     protected final OperatorDescriptorId odId;
 
-    protected PartitionConstraint partitionConstraint;
-
     protected String[] partitions;
 
     protected final RecordDescriptor[] recordDescriptors;
@@ -64,21 +63,16 @@
     }
 
     @Override
-    public PartitionConstraint getPartitionConstraint() {
-        return partitionConstraint;
-    }
-
-    @Override
-    public void setPartitionConstraint(PartitionConstraint partitionConstraint) {
-        this.partitionConstraint = partitionConstraint;
-    }
-
-    @Override
     public RecordDescriptor[] getOutputRecordDescriptors() {
         return recordDescriptors;
     }
 
     @Override
+    public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan) {
+        // do nothing
+    }
+
+    @Override
     public JSONObject toJSON() throws JSONException {
         JSONObject jop = new JSONObject();
         jop.put("type", "operator");
@@ -86,7 +80,6 @@
         jop.put("java-class", getClass().getName());
         jop.put("in-arity", getInputArity());
         jop.put("out-arity", getOutputArity());
-        jop.put("partition-constraint", String.valueOf(partitionConstraint));
         return jop;
     }
 }
\ 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 e1154e4..a038a40 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
@@ -17,10 +17,15 @@
 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.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.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.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.NonDeterministicFrameReader;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
@@ -45,4 +50,15 @@
             throws HyracksDataException {
         return new NonDeterministicFrameReader(ctx, demux);
     }
+
+    @Override
+    public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan) {
+        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));
+    }
 }
\ No newline at end of file
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 07fd5ba..b5468ad 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
@@ -22,10 +22,7 @@
 
 import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
@@ -128,9 +125,7 @@
         DataGenOperatorDescriptor dataGen = new DataGenOperatorDescriptor(spec, recDesc, options.numTuples, 2, 0,
                 100000, 10, 100);
         // run data generator on first nodecontroller given
-        PartitionConstraint dataGenConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(splitNCs[0]) });
-        dataGen.setPartitionConstraint(dataGenConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dataGen, splitNCs[0]);
 
         IBTreeRegistryProvider btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
         IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
@@ -164,8 +159,7 @@
                 recDesc, storageManager, btreeRegistryProvider, primarySplitProvider, primaryInteriorFrameFactory,
                 primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, primaryFieldPermutation,
                 BTreeOp.BTO_INSERT);
-        PartitionConstraint primaryInsertConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        primaryInsert.setPartitionConstraint(primaryInsertConstraint);
+        JobHelper.createPartitionConstraint(spec, primaryInsert, splitNCs);
 
         // prepare insertion into secondary index
         // tuples to be put into B-Tree shall have 2 fields
@@ -194,13 +188,11 @@
                 recDesc, storageManager, btreeRegistryProvider, secondarySplitProvider, secondaryInteriorFrameFactory,
                 secondaryLeafFrameFactory, secondaryTypeTraits, secondaryComparatorFactories,
                 secondaryFieldPermutation, BTreeOp.BTO_INSERT);
-        PartitionConstraint secondaryInsertConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        secondaryInsert.setPartitionConstraint(secondaryInsertConstraint);
+        JobHelper.createPartitionConstraint(spec, secondaryInsert, splitNCs);
 
         // end the insert pipeline at this sink operator
         NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
-        PartitionConstraint nullSinkPartitionConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        nullSink.setPartitionConstraint(nullSinkPartitionConstraint);
+        JobHelper.createPartitionConstraint(spec, nullSink, splitNCs);
 
         // distribute the records from the datagen via hashing to the bulk load
         // ops
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/JobHelper.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/JobHelper.java
index af79d32..63f07f0 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/JobHelper.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/JobHelper.java
@@ -17,11 +17,10 @@
 
 import java.io.File;
 
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -37,11 +36,7 @@
         return splitProvider;
     }
 
-    public static PartitionConstraint createPartitionConstraint(String[] splitNCs) {
-        LocationConstraint[] lConstraints = new LocationConstraint[splitNCs.length];
-        for (int i = 0; i < splitNCs.length; ++i) {
-            lConstraints[i] = new AbsoluteLocationConstraint(splitNCs[i]);
-        }
-        return new ExplicitPartitionConstraint(lConstraints);
+    public static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, String[] splitNCs) {
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, splitNCs);
     }
 }
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 336efb3..204bdf5 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
@@ -21,10 +21,7 @@
 
 import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
@@ -125,9 +122,7 @@
         DataGenOperatorDescriptor dataGen = new DataGenOperatorDescriptor(spec, recDesc, options.numTuples, 2, 0,
                 100000, 10, 50);
         // run data generator on first nodecontroller given
-        PartitionConstraint dataGenConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(splitNCs[0]) });
-        dataGen.setPartitionConstraint(dataGenConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dataGen, splitNCs[0]);
 
         // sort the tuples as preparation for bulk load
         // fields to sort on
@@ -137,8 +132,7 @@
         comparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, options.sbSize, sortFields,
                 comparatorFactories, recDesc);
-        PartitionConstraint sorterConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        sorter.setPartitionConstraint(sorterConstraint);
+        JobHelper.createPartitionConstraint(spec, sorter, splitNCs);
 
         // tuples to be put into B-Tree shall have 4 fields
         int fieldCount = 4;
@@ -164,8 +158,7 @@
         BTreeBulkLoadOperatorDescriptor btreeBulkLoad = new BTreeBulkLoadOperatorDescriptor(spec, storageManager,
                 btreeRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory, typeTraits,
                 comparatorFactories, fieldPermutation, 0.7f);
-        PartitionConstraint bulkLoadConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        btreeBulkLoad.setPartitionConstraint(bulkLoadConstraint);
+        JobHelper.createPartitionConstraint(spec, btreeBulkLoad, splitNCs);
 
         // distribute the records from the datagen via hashing to the bulk load
         // ops
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 fedf0bf..a7ee0dc 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
@@ -21,7 +21,6 @@
 
 import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -112,8 +111,7 @@
         BTreeFileEnlistmentOperatorDescriptor fileEnlistmentOp = new BTreeFileEnlistmentOperatorDescriptor(spec,
                 recDesc, storageManager, btreeRegistryProvider, btreeSplitProvider, interiorFrameFactory,
                 leafFrameFactory, typeTraits, comparatorFactories);
-        PartitionConstraint fileEnlistmentConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        fileEnlistmentOp.setPartitionConstraint(fileEnlistmentConstraint);
+        JobHelper.createPartitionConstraint(spec, fileEnlistmentOp, splitNCs);
 
         spec.addRoot(fileEnlistmentOp);
 
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 fe9bee2..b43efd2 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
@@ -22,7 +22,6 @@
 
 import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -137,8 +136,7 @@
 
         ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
                 keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraint keyProviderPartitionConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        keyProviderOp.setPartitionConstraint(keyProviderPartitionConstraint);
+        JobHelper.createPartitionConstraint(spec, keyProviderOp, splitNCs);
 
         int[] lowKeyFields = { 0 }; // low key is in field 0 of tuples going
                                     // into search op
@@ -149,13 +147,11 @@
         BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(spec, recDesc, storageManager,
                 btreeRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory, typeTraits,
                 comparatorFactories, true, lowKeyFields, highKeyFields, true, true);
-        PartitionConstraint btreeSearchConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        btreeSearchOp.setPartitionConstraint(btreeSearchConstraint);
+        JobHelper.createPartitionConstraint(spec, btreeSearchOp, splitNCs);
 
         // have each node print the results of its respective B-Tree
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        printer.setPartitionConstraint(printerConstraint);
+        JobHelper.createPartitionConstraint(spec, printer, splitNCs);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, btreeSearchOp, 0);
 
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 403e0a9..4940c2a 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
@@ -21,7 +21,6 @@
 
 import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -127,8 +126,7 @@
         BTreeDiskOrderScanOperatorDescriptor btreeScanOp = new BTreeDiskOrderScanOperatorDescriptor(spec, recDesc,
                 storageManager, btreeRegistryProvider, primarySplitProvider, primaryInteriorFrameFactory,
                 primaryLeafFrameFactory, primaryTypeTraits);
-        PartitionConstraint scanPartitionConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        btreeScanOp.setPartitionConstraint(scanPartitionConstraint);
+        JobHelper.createPartitionConstraint(spec, btreeScanOp, splitNCs);
 
         // sort the tuples as preparation for bulk load into secondary index
         // fields to sort on
@@ -139,8 +137,7 @@
         comparatorFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, options.sbSize, sortFields,
                 comparatorFactories, recDesc);
-        PartitionConstraint sorterConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        sorter.setPartitionConstraint(sorterConstraint);
+        JobHelper.createPartitionConstraint(spec, sorter, splitNCs);
 
         // tuples to be put into B-Tree shall have 2 fields
         int secondaryFieldCount = 2;
@@ -161,8 +158,7 @@
         BTreeBulkLoadOperatorDescriptor btreeBulkLoad = new BTreeBulkLoadOperatorDescriptor(spec, storageManager,
                 btreeRegistryProvider, btreeSplitProvider, secondaryInteriorFrameFactory, secondaryLeafFrameFactory,
                 secondaryTypeTraits, comparatorFactories, fieldPermutation, 0.7f);
-        PartitionConstraint bulkLoadConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        btreeBulkLoad.setPartitionConstraint(bulkLoadConstraint);
+        JobHelper.createPartitionConstraint(spec, btreeBulkLoad, splitNCs);
 
         // connect the ops
 
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 ba2c72f..a770f09 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
@@ -22,7 +22,6 @@
 
 import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -159,8 +158,7 @@
 
         ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
                 keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraint keyProviderPartitionConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        keyProviderOp.setPartitionConstraint(keyProviderPartitionConstraint);
+        JobHelper.createPartitionConstraint(spec, keyProviderOp, splitNCs);
 
         int[] secondaryLowKeyFields = { 0 }; // low key is in field 0 of tuples
                                              // going into secondary index
@@ -175,8 +173,7 @@
                 storageManager, btreeRegistryProvider, secondarySplitProvider, secondaryInteriorFrameFactory,
                 secondaryLeafFrameFactory, secondaryTypeTraits, comparatorFactories, true, secondaryLowKeyFields,
                 secondaryHighKeyFields, true, true);
-        PartitionConstraint secondarySearchConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        secondarySearchOp.setPartitionConstraint(secondarySearchConstraint);
+        JobHelper.createPartitionConstraint(spec, secondarySearchOp, splitNCs);
 
         // secondary index will output tuples with [UTF8String, Integer]
         // the Integer field refers to the key in the primary index of the
@@ -192,13 +189,11 @@
                 storageManager, btreeRegistryProvider, primarySplitProvider, primaryInteriorFrameFactory,
                 primaryLeafFrameFactory, primaryTypeTraits, comparatorFactories, true, primaryLowKeyFields,
                 primaryHighKeyFields, true, true);
-        PartitionConstraint primarySearchConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        primarySearchOp.setPartitionConstraint(primarySearchConstraint);
+        JobHelper.createPartitionConstraint(spec, primarySearchOp, splitNCs);
 
         // have each node print the results of its respective B-Tree
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerConstraint = JobHelper.createPartitionConstraint(splitNCs);
-        printer.setPartitionConstraint(printerConstraint);
+        JobHelper.createPartitionConstraint(spec, printer, splitNCs);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondarySearchOp, 0);
 
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeOperatorsTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeOperatorsTest.java
index 7f2152e..33aebc1 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeOperatorsTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeOperatorsTest.java
@@ -20,10 +20,7 @@
 
 import org.junit.Test;
 
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 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;
@@ -99,15 +96,11 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 0 },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
-        PartitionConstraint sortersPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        sorter.setPartitionConstraint(sortersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
 
         // declare fields
         int fieldCount = 3;
@@ -138,9 +131,7 @@
         BTreeBulkLoadOperatorDescriptor btreeBulkLoad = new BTreeBulkLoadOperatorDescriptor(spec, storageManager,
                 btreeRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory, typeTraits,
                 comparatorFactories, fieldPermutation, 0.7f);
-        PartitionConstraint btreePartitionConstraintA = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        btreeBulkLoad.setPartitionConstraint(btreePartitionConstraintA);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, btreeBulkLoad, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
 
@@ -234,9 +225,7 @@
 
         ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
                 keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraint keyProviderPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        keyProviderOp.setPartitionConstraint(keyProviderPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
         IBTreeRegistryProvider btreeRegistryProvider = new TestBTreeRegistryProvider();
 
         RecordDescriptor recDesc = new RecordDescriptor(new ISerializerDeserializer[] {
@@ -257,14 +246,10 @@
         // bufferCacheProvider, btreeRegistryProvider, 0, "btreetest.bin",
         // interiorFrameFactory, leafFrameFactory, cmp);
 
-        PartitionConstraint btreePartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        btreeSearchOp.setPartitionConstraint(btreePartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, btreeSearchOp, NC1_ID);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, btreeSearchOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), btreeSearchOp, 0, printer, 0);
@@ -296,9 +281,7 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         // we will create a primary index and 2 secondary indexes
         // first create comparators for primary index
@@ -406,9 +389,7 @@
                 ordersDesc, storageManager, btreeRegistryProvider, btreeSplitProviderA, primaryInteriorFrameFactory,
                 primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, fieldPermutationA,
                 BTreeOp.BTO_INSERT);
-        PartitionConstraint insertPartitionConstraintA = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        insertOpA.setPartitionConstraint(insertPartitionConstraintA);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, insertOpA, NC1_ID);
 
         // first secondary index
         IFileSplitProvider btreeSplitProviderB = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -418,9 +399,7 @@
                 ordersDesc, storageManager, btreeRegistryProvider, btreeSplitProviderB, secondaryInteriorFrameFactory,
                 secondaryLeafFrameFactory, secondaryTypeTraits, secondaryComparatorFactories, fieldPermutationB,
                 BTreeOp.BTO_INSERT);
-        PartitionConstraint insertPartitionConstraintB = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        insertOpB.setPartitionConstraint(insertPartitionConstraintB);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, insertOpB, NC1_ID);
 
         // second secondary index
         IFileSplitProvider btreeSplitProviderC = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -430,14 +409,10 @@
                 ordersDesc, storageManager, btreeRegistryProvider, btreeSplitProviderC, secondaryInteriorFrameFactory,
                 secondaryLeafFrameFactory, secondaryTypeTraits, secondaryComparatorFactories, fieldPermutationC,
                 BTreeOp.BTO_INSERT);
-        PartitionConstraint insertPartitionConstraintC = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        insertOpC.setPartitionConstraint(insertPartitionConstraintC);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, insertOpC, NC1_ID);
 
         NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
-        PartitionConstraint nullSinkPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        nullSink.setPartitionConstraint(nullSinkPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, nullSink, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, insertOpA, 0);
 
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 858ff6c..2bb1d80 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
@@ -43,7 +43,6 @@
         CCConfig ccConfig = new CCConfig();
         ccConfig.port = 39001;
         ccConfig.profileDumpPeriod = 1000;
-        ccConfig.useJOL = true;
         cc = new ClusterControllerService(ccConfig);
         cc.start();
 
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 e5d286e..8468b54 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
@@ -18,10 +18,7 @@
 
 import org.junit.Test;
 
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
@@ -68,15 +65,11 @@
                 splitProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE }, ','),
                 desc);
-        PartitionConstraint csvPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        csvScanner.setPartitionConstraint(csvPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC1_ID);
 
         InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 0 },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, desc);
-        PartitionConstraint sorterPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        sorter.setPartitionConstraint(sorterPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
 
         RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
@@ -86,29 +79,21 @@
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
                 new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                 desc2);
-        PartitionConstraint groupPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        group.setPartitionConstraint(groupPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID);
 
         InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
                 new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, desc2);
-        PartitionConstraint sorterPartitionConstraint2 = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        sorter2.setPartitionConstraint(sorterPartitionConstraint2);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter2, NC1_ID);
 
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
                 new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
                         new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), desc3);
-        PartitionConstraint groupPartitionConstraint2 = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        group2.setPartitionConstraint(groupPartitionConstraint2);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
@@ -147,16 +132,11 @@
                 splitProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE }, ','),
                 desc);
-        PartitionConstraint csvPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        csvScanner.setPartitionConstraint(csvPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC1_ID);
 
         InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 0 },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, desc);
-        PartitionConstraint sorterPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID),
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        sorter.setPartitionConstraint(sorterPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
         RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
@@ -166,30 +146,21 @@
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
                 new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                 desc2);
-        PartitionConstraint groupPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID),
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        group.setPartitionConstraint(groupPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
         InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
                 new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, desc2);
-        PartitionConstraint sorterPartitionConstraint2 = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        sorter2.setPartitionConstraint(sorterPartitionConstraint2);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter2, NC1_ID, NC2_ID);
 
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
                 new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
                         new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), desc3);
-        PartitionConstraint groupPartitionConstraint2 = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        group2.setPartitionConstraint(groupPartitionConstraint2);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
@@ -228,16 +199,11 @@
                 splitProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE }, ','),
                 desc);
-        PartitionConstraint csvPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        csvScanner.setPartitionConstraint(csvPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC1_ID);
 
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 3, new int[] { 0 },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, desc);
-        PartitionConstraint sorterPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID),
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        sorter.setPartitionConstraint(sorterPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
         RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
@@ -247,30 +213,21 @@
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
                 new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                 desc2);
-        PartitionConstraint groupPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID),
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        group.setPartitionConstraint(groupPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
         InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
                 new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, desc2);
-        PartitionConstraint sorterPartitionConstraint2 = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        sorter2.setPartitionConstraint(sorterPartitionConstraint2);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter2, NC1_ID, NC2_ID);
 
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
                 new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
                         new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), desc3);
-        PartitionConstraint groupPartitionConstraint2 = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        group2.setPartitionConstraint(groupPartitionConstraint2);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 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 5ab05cc..f0974d9 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
@@ -18,10 +18,7 @@
 
 import org.junit.Test;
 
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -62,14 +59,10 @@
                 splitProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE }, ','),
                 desc);
-        PartitionConstraint csvPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC2_ID), new AbsoluteLocationConstraint(NC1_ID) });
-        csvScanner.setPartitionConstraint(csvPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID, NC1_ID);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC2_ID), new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn, csvScanner, 0, printer, 0);
@@ -98,14 +91,10 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 0 },
@@ -136,14 +125,10 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 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 f2b63ae..b0c33ae 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
@@ -18,10 +18,7 @@
 
 import org.junit.Test;
 
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -67,20 +64,14 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
 
         InMemorySortOperatorDescriptor sorter = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
-        PartitionConstraint sortersPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        sorter.setPartitionConstraint(sortersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
 
@@ -113,21 +104,15 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
 
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 4, new int[] { 1, 0 },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
                         UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
-        PartitionConstraint sortersPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        sorter.setPartitionConstraint(sortersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
 
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 452b840..4958a35 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
@@ -18,11 +18,7 @@
 
 import org.junit.Test;
 
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionCountConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -99,9 +95,7 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
@@ -109,22 +103,16 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
-        PartitionConstraint custPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        custScanner.setPartitionConstraint(custPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
         InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
                 new int[] { 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
-        PartitionConstraint joinPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        join.setPartitionConstraint(joinPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         IOperatorDescriptor printer = DEBUG ? new PrinterOperatorDescriptor(spec)
                 : new NullSinkOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
@@ -179,9 +167,7 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
@@ -189,23 +175,17 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
-        PartitionConstraint custPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        custScanner.setPartitionConstraint(custPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
         GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(spec, 4, 10, 200, 1.2,
                 new int[] { 1 }, new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc);
-        PartitionConstraint joinPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        join.setPartitionConstraint(joinPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         IOperatorDescriptor printer = DEBUG ? new PrinterOperatorDescriptor(spec)
                 : new NullSinkOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
@@ -260,9 +240,7 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
@@ -270,23 +248,17 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
-        PartitionConstraint custPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        custScanner.setPartitionConstraint(custPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
         HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(spec, 5, 20, 200, 1.2,
                 new int[] { 1 }, new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc);
-        PartitionConstraint joinPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        join.setPartitionConstraint(joinPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         IOperatorDescriptor printer = DEBUG ? new PrinterOperatorDescriptor(spec)
                 : new NullSinkOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
@@ -343,9 +315,7 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
@@ -353,22 +323,16 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
-        PartitionConstraint custPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        custScanner.setPartitionConstraint(custPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
         InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
                 new int[] { 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
-        PartitionConstraint joinPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        join.setPartitionConstraint(joinPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         IOperatorDescriptor printer = DEBUG ? new PrinterOperatorDescriptor(spec)
                 : new NullSinkOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
@@ -429,9 +393,7 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
@@ -439,23 +401,17 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
-        PartitionConstraint custPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        custScanner.setPartitionConstraint(custPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
         GraceHashJoinOperatorDescriptor join = new GraceHashJoinOperatorDescriptor(spec, 3, 20, 100, 1.2,
                 new int[] { 1 }, new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc);
-        PartitionConstraint joinPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        join.setPartitionConstraint(joinPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         IOperatorDescriptor printer = DEBUG ? new PrinterOperatorDescriptor(spec)
                 : new NullSinkOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
@@ -516,9 +472,7 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
@@ -526,23 +480,17 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
-        PartitionConstraint custPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        custScanner.setPartitionConstraint(custPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
         HybridHashJoinOperatorDescriptor join = new HybridHashJoinOperatorDescriptor(spec, 3, 20, 100, 1.2,
                 new int[] { 1 }, new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc);
-        PartitionConstraint joinPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        join.setPartitionConstraint(joinPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         IOperatorDescriptor printer = DEBUG ? new PrinterOperatorDescriptor(spec)
                 : new NullSinkOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
@@ -603,9 +551,7 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
@@ -613,20 +559,16 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
-        PartitionConstraint custPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        custScanner.setPartitionConstraint(custPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
         InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
                 new int[] { 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
-        join.setPartitionConstraint(new PartitionCountConstraint(2));
+        PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
 
         IOperatorDescriptor printer = DEBUG ? new PrinterOperatorDescriptor(spec)
                 : new NullSinkOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
@@ -687,9 +629,7 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
@@ -697,30 +637,22 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
-        PartitionConstraint custPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        custScanner.setPartitionConstraint(custPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
 
         MaterializingOperatorDescriptor ordMat = new MaterializingOperatorDescriptor(spec, ordersDesc);
-        ordMat.setPartitionConstraint(new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) }));
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordMat, NC1_ID, NC2_ID);
 
         MaterializingOperatorDescriptor custMat = new MaterializingOperatorDescriptor(spec, custDesc);
-        custMat.setPartitionConstraint(new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) }));
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custMat, NC1_ID, NC2_ID);
 
         InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 1 },
                 new int[] { 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc, 128);
-        PartitionConstraint joinPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC1_ID), new AbsoluteLocationConstraint(NC2_ID) });
-        join.setPartitionConstraint(joinPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         IOperatorDescriptor printer = DEBUG ? new PrinterOperatorDescriptor(spec)
                 : new NullSinkOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordPartConn = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
index bdf8ebb..2cfe438 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
@@ -4,10 +4,7 @@
 
 import org.junit.Test;
 
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 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;
@@ -44,9 +41,7 @@
         FileScanOperatorDescriptor dblpTitleScanner = new FileScanOperatorDescriptor(spec, dblpTitleSplitProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE }, '|'), dblpTitleRecDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        dblpTitleScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dblpTitleScanner, NC1_ID);
 
         RecordDescriptor tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
@@ -55,14 +50,10 @@
         int[] projFields = { 0 };
         BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec,
                 tokenizerRecDesc, tokenizerFactory, tokenFields, projFields);
-        PartitionConstraint tokenizerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        binaryTokenizer.setPartitionConstraint(tokenizerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 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 2621049..694f183 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
@@ -18,10 +18,7 @@
 
 import org.junit.Test;
 
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
@@ -91,9 +88,7 @@
                 splitProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE }, ','),
                 desc);
-        PartitionConstraint csvPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC2_ID), new AbsoluteLocationConstraint(NC1_ID) });
-        csvScanner.setPartitionConstraint(csvPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID, NC1_ID);
 
         int[] keys = new int[] { 0 };
         int tableSize = 8;
@@ -106,18 +101,16 @@
                 // Hash partitioner
                 new FieldHashPartitionComputerFactory(keys,
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                        // Key comparator
-                        new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                        // Aggregator factory
-                        new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
-                        outputRec, // Output format
-                        tableSize // Size of the hashing table, which is used to control
-                        // the partition when hashing
+                // Key comparator
+                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                // Aggregator factory
+                new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
+                outputRec, // Output format
+                tableSize // Size of the hashing table, which is used to control
+        // the partition when hashing
         );
 
-        PartitionConstraint grouperPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC2_ID), new AbsoluteLocationConstraint(NC1_ID) });
-        grouper.setPartitionConstraint(grouperPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keys,
@@ -125,9 +118,7 @@
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC2_ID), new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -161,9 +152,7 @@
                 splitProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE }, ','),
                 desc);
-        PartitionConstraint csvPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC2_ID), new AbsoluteLocationConstraint(NC1_ID) });
-        csvScanner.setPartitionConstraint(csvPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID, NC1_ID);
 
         int[] keys = new int[] { 0 };
         int tableSize = 8;
@@ -173,13 +162,11 @@
                 keys,
                 new FieldHashPartitionComputerFactory(keys,
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                        new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                        new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
-                        outputRec, tableSize);
+                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
+                outputRec, tableSize);
 
-        PartitionConstraint grouperPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC2_ID), new AbsoluteLocationConstraint(NC1_ID) });
-        grouper.setPartitionConstraint(grouperPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keys,
@@ -187,9 +174,7 @@
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(new LocationConstraint[] {
-                new AbsoluteLocationConstraint(NC2_ID), new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -208,7 +193,7 @@
         JobSpecification spec = new JobSpecification();
 
         FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
-        "data/tpch0.001/lineitem.tbl"))) };
+                "data/tpch0.001/lineitem.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
@@ -229,31 +214,23 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
-        PartitionConstraint csvPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(csvPartitionConstraint);
-
         int[] keys = new int[] { 0 };
         int tableSize = 8;
 
         ExternalHashGroupOperatorDescriptor grouper = new ExternalHashGroupOperatorDescriptor(spec, keys, 3, false,
                 new FieldHashPartitionComputerFactory(keys,
                         new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                        new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
-                        new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory(),
-                                new SumAggregatorFactory(4), new MinMaxAggregatorFactory(true, 5) }), outputRec, tableSize);
+                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
+                new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory(),
+                        new SumAggregatorFactory(4), new MinMaxAggregatorFactory(true, 5) }), outputRec, tableSize);
 
-        PartitionConstraint grouperPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        grouper.setPartitionConstraint(grouperPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keys,
@@ -261,9 +238,7 @@
         spec.connect(conn1, ordScanner, 0, grouper, 0);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -282,7 +257,7 @@
         JobSpecification spec = new JobSpecification();
 
         FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
-        "data/tpch0.001/lineitem.tbl"))) };
+                "data/tpch0.001/lineitem.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -303,19 +278,13 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
                 FloatSerializerDeserializer.INSTANCE });
 
-        PartitionConstraint csvPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(csvPartitionConstraint);
-
         // Group on two fields
         int[] keys = new int[] { 0, 1 };
         int tableSize = 8;
@@ -323,15 +292,13 @@
         ExternalHashGroupOperatorDescriptor grouper = new ExternalHashGroupOperatorDescriptor(spec, keys, 3, false,
                 new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
                         UTF8StringBinaryHashFunctionFactory.INSTANCE, UTF8StringBinaryHashFunctionFactory.INSTANCE }),
-                        new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
-            UTF8StringBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
-                    new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory(),
-                            new SumAggregatorFactory(4), new MinMaxAggregatorFactory(true, 5) }), outputRec,
-                            tableSize);
+                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
+                        UTF8StringBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
+                        new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory(),
+                                new SumAggregatorFactory(4), new MinMaxAggregatorFactory(true, 5) }), outputRec,
+                tableSize);
 
-        PartitionConstraint grouperPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        grouper.setPartitionConstraint(grouperPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
@@ -339,9 +306,7 @@
         spec.connect(conn1, ordScanner, 0, grouper, 0);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -360,7 +325,7 @@
         JobSpecification spec = new JobSpecification();
 
         FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
-        "data/tpch0.001/lineitem.tbl"))) };
+                "data/tpch0.001/lineitem.tbl"))) };
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
@@ -381,19 +346,13 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, }, '|'), ordersDesc);
-        PartitionConstraint ordersPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(ordersPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
                 FloatSerializerDeserializer.INSTANCE });
 
-        PartitionConstraint csvPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        ordScanner.setPartitionConstraint(csvPartitionConstraint);
-
         // Group on two fields
         int[] keys = new int[] { 0, 1 };
         int tableSize = 8;
@@ -401,15 +360,13 @@
         ExternalHashGroupOperatorDescriptor grouper = new ExternalHashGroupOperatorDescriptor(spec, keys, 3000, true,
                 new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
                         IntegerBinaryHashFunctionFactory.INSTANCE, IntegerBinaryHashFunctionFactory.INSTANCE }),
-                        new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE,
-            IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
-                    new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory(),
-                            new SumAggregatorFactory(4), new MinMaxAggregatorFactory(true, 5) }), outputRec,
-                            tableSize);
+                new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE,
+                        IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
+                        new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory(),
+                                new SumAggregatorFactory(4), new MinMaxAggregatorFactory(true, 5) }), outputRec,
+                tableSize);
 
-        PartitionConstraint grouperPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        grouper.setPartitionConstraint(grouperPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
@@ -417,9 +374,7 @@
         spec.connect(conn1, ordScanner, 0, grouper, 0);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        PartitionConstraint printerPartitionConstraint = new ExplicitPartitionConstraint(
-                new LocationConstraint[] { new AbsoluteLocationConstraint(NC1_ID) });
-        printer.setPartitionConstraint(printerPartitionConstraint);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -427,4 +382,4 @@
         spec.addRoot(printer);
         runTest(spec);
     }
-}
+}
\ No newline at end of file
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 f02c855..8d22bca 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
@@ -22,11 +22,9 @@
 
 import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -179,7 +177,7 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, }, '|'), inDesc);
 
-        fileScanner.setPartitionConstraint(createPartitionConstraint(inSplits));
+        createPartitionConstraint(spec, fileScanner, inSplits);
 
         // Output: each unique string with an integer count
         RecordDescriptor outDesc = new RecordDescriptor(new ISerializerDeserializer[] {
@@ -190,7 +188,7 @@
         // Specify the grouping key, which will be the string extracted during
         // the scan.
         int[] keys = new int[] { 0,
-                // 1
+        // 1
         };
 
         AbstractOperatorDescriptor grouper;
@@ -199,19 +197,19 @@
             case 0: // External hash group
                 grouper = new ExternalHashGroupOperatorDescriptor(spec, keys, framesLimit, false,
                         new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                                // IntegerBinaryHashFunctionFactory.INSTANCE,
+                        // IntegerBinaryHashFunctionFactory.INSTANCE,
                                 IntegerBinaryHashFunctionFactory.INSTANCE }), new IBinaryComparatorFactory[] {
-                    // IntegerBinaryComparatorFactory.INSTANCE,
-                    IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
-                            new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), outDesc,
-                            htSize);
+                        // IntegerBinaryComparatorFactory.INSTANCE,
+                        IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
+                                new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), outDesc,
+                        htSize);
 
-                grouper.setPartitionConstraint(createPartitionConstraint(outSplits));
+                createPartitionConstraint(spec, grouper, outSplits);
 
                 // Connect scanner with the grouper
                 IConnectorDescriptor scanGroupConn = new MToNHashPartitioningConnectorDescriptor(spec,
                         new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                                // IntegerBinaryHashFunctionFactory.INSTANCE,
+                        // IntegerBinaryHashFunctionFactory.INSTANCE,
                                 IntegerBinaryHashFunctionFactory.INSTANCE }));
                 spec.connect(scanGroupConn, fileScanner, 0, grouper, 0);
                 break;
@@ -220,21 +218,21 @@
                         new IBinaryComparatorFactory[] {
                         // IntegerBinaryComparatorFactory.INSTANCE,
                         IntegerBinaryComparatorFactory.INSTANCE }, inDesc);
-                sorter.setPartitionConstraint(createPartitionConstraint(inSplits));
+                createPartitionConstraint(spec, sorter, inSplits);
 
                 // Connect scan operator with the sorter
                 IConnectorDescriptor scanSortConn = new MToNHashPartitioningConnectorDescriptor(spec,
                         new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                                // IntegerBinaryHashFunctionFactory.INSTANCE,
+                        // IntegerBinaryHashFunctionFactory.INSTANCE,
                                 IntegerBinaryHashFunctionFactory.INSTANCE }));
                 spec.connect(scanSortConn, fileScanner, 0, sorter, 0);
 
                 grouper = new PreclusteredGroupOperatorDescriptor(spec, keys, new IBinaryComparatorFactory[] {
-                        // IntegerBinaryComparatorFactory.INSTANCE,
+                // IntegerBinaryComparatorFactory.INSTANCE,
                         IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
                                 new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), outDesc);
 
-                grouper.setPartitionConstraint(createPartitionConstraint(outSplits));
+                createPartitionConstraint(spec, grouper, outSplits);
 
                 // Connect sorter with the pre-cluster
                 OneToOneConnectorDescriptor sortGroupConn = new OneToOneConnectorDescriptor(spec);
@@ -245,36 +243,36 @@
                         new IBinaryHashFunctionFactory[] {
                         // IntegerBinaryHashFunctionFactory.INSTANCE,
                         IntegerBinaryHashFunctionFactory.INSTANCE }), new IBinaryComparatorFactory[] {
-                    // IntegerBinaryComparatorFactory.INSTANCE,
-                    IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
-                            new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), outDesc,
-                            htSize);
+                // IntegerBinaryComparatorFactory.INSTANCE,
+                        IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
+                                new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), outDesc,
+                        htSize);
 
-                grouper.setPartitionConstraint(createPartitionConstraint(outSplits));
+                createPartitionConstraint(spec, grouper, outSplits);
 
                 // Connect scanner with the grouper
                 IConnectorDescriptor scanConn = new MToNHashPartitioningConnectorDescriptor(spec,
                         new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                                // IntegerBinaryHashFunctionFactory.INSTANCE,
+                        // IntegerBinaryHashFunctionFactory.INSTANCE,
                                 IntegerBinaryHashFunctionFactory.INSTANCE }));
                 spec.connect(scanConn, fileScanner, 0, grouper, 0);
                 break;
             default:
                 grouper = new ExternalHashGroupOperatorDescriptor(spec, keys, framesLimit, false,
                         new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                                // IntegerBinaryHashFunctionFactory.INSTANCE,
+                        // IntegerBinaryHashFunctionFactory.INSTANCE,
                                 IntegerBinaryHashFunctionFactory.INSTANCE }), new IBinaryComparatorFactory[] {
-                    // IntegerBinaryComparatorFactory.INSTANCE,
-                    IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
-                            new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), outDesc,
-                            htSize);
+                        // IntegerBinaryComparatorFactory.INSTANCE,
+                        IntegerBinaryComparatorFactory.INSTANCE }, new MultiAggregatorFactory(
+                                new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), outDesc,
+                        htSize);
 
-                grouper.setPartitionConstraint(createPartitionConstraint(outSplits));
+                createPartitionConstraint(spec, grouper, outSplits);
 
                 // Connect scanner with the grouper
                 IConnectorDescriptor scanGroupConnDef = new MToNHashPartitioningConnectorDescriptor(spec,
                         new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                                // IntegerBinaryHashFunctionFactory.INSTANCE,
+                        // IntegerBinaryHashFunctionFactory.INSTANCE,
                                 IntegerBinaryHashFunctionFactory.INSTANCE }));
                 spec.connect(scanGroupConnDef, fileScanner, 0, grouper, 0);
         }
@@ -288,7 +286,7 @@
         else
             writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
 
-        writer.setPartitionConstraint(createPartitionConstraint(outSplits));
+        createPartitionConstraint(spec, writer, outSplits);
 
         IConnectorDescriptor groupOutConn = new OneToOneConnectorDescriptor(spec);
         spec.connect(groupOutConn, grouper, 0, writer, 0);
@@ -297,11 +295,11 @@
         return spec;
     }
 
-    private static PartitionConstraint createPartitionConstraint(FileSplit[] splits) {
-        LocationConstraint[] lConstraints = new LocationConstraint[splits.length];
+    private static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
+        String[] parts = new String[splits.length];
         for (int i = 0; i < splits.length; ++i) {
-            lConstraints[i] = new AbsoluteLocationConstraint(splits[i].getNodeName());
+            parts[i] = splits[i].getNodeName();
         }
-        return new ExplicitPartitionConstraint(lConstraints);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, parts);
     }
-}
+}
\ No newline at end of file
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 84cbd9a..4f20774 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
@@ -22,10 +22,7 @@
 
 import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -125,7 +122,7 @@
 
         FileScanOperatorDescriptor wordScanner = new FileScanOperatorDescriptor(spec, splitsProvider,
                 new WordTupleParserFactory(), wordDesc);
-        wordScanner.setPartitionConstraint(createPartitionConstraint(inSplits));
+        createPartitionConstraint(spec, wordScanner, inSplits);
 
         RecordDescriptor groupResultDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
@@ -139,7 +136,7 @@
                     new MultiAggregatorFactory(
                             new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                     groupResultDesc, htSize);
-            gBy.setPartitionConstraint(createPartitionConstraint(outSplits));
+            createPartitionConstraint(spec, gBy, outSplits);
             IConnectorDescriptor scanGroupConn = new MToNHashPartitioningConnectorDescriptor(spec,
                     new FieldHashPartitionComputerFactory(keys,
                             new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
@@ -150,7 +147,7 @@
                     keys, new UTF8StringNormalizedKeyComputerFactory(), cfs, wordDesc)
                     : new ExternalSortOperatorDescriptor(spec, sbSize, keys,
                             new UTF8StringNormalizedKeyComputerFactory(), cfs, wordDesc);
-            sorter.setPartitionConstraint(createPartitionConstraint(outSplits));
+            createPartitionConstraint(spec, sorter, outSplits);
 
             IConnectorDescriptor scanSortConn = new MToNHashPartitioningConnectorDescriptor(spec,
                     new FieldHashPartitionComputerFactory(keys,
@@ -162,14 +159,14 @@
                     new MultiAggregatorFactory(
                             new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                     groupResultDesc);
-            gBy.setPartitionConstraint(createPartitionConstraint(outSplits));
+            createPartitionConstraint(spec, gBy, outSplits);
             OneToOneConnectorDescriptor sortGroupConn = new OneToOneConnectorDescriptor(spec);
             spec.connect(sortGroupConn, sorter, 0, gBy, 0);
         }
 
         IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(outSplits);
         FrameFileWriterOperatorDescriptor writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
-        writer.setPartitionConstraint(createPartitionConstraint(outSplits));
+        createPartitionConstraint(spec, writer, outSplits);
 
         IConnectorDescriptor gbyPrinterConn = new OneToOneConnectorDescriptor(spec);
         spec.connect(gbyPrinterConn, gBy, 0, writer, 0);
@@ -178,11 +175,11 @@
         return spec;
     }
 
-    private static PartitionConstraint createPartitionConstraint(FileSplit[] splits) {
-        LocationConstraint[] lConstraints = new LocationConstraint[splits.length];
+    private static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
+        String[] parts = new String[splits.length];
         for (int i = 0; i < splits.length; ++i) {
-            lConstraints[i] = new AbsoluteLocationConstraint(splits[i].getNodeName());
+            parts[i] = splits[i].getNodeName();
         }
-        return new ExplicitPartitionConstraint(lConstraints);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, parts);
     }
 }
\ No newline at end of file
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 e26d2a1..b1a1224 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
@@ -1,22 +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.examples.tpch.client;
 
 import java.io.File;
+import java.util.EnumSet;
 import java.util.UUID;
 
+import org.kohsuke.args4j.CmdLineParser;
+import org.kohsuke.args4j.Option;
+
 import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.AbsoluteLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ChoiceLocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.LocationConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionCountConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobFlag;
 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;
@@ -34,51 +49,84 @@
 import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.FrameFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.group.HashGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.InMemoryHashJoinOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
 
 public class Main {
-    public static void main(String[] args) throws Exception {
-        String appName = args[0];
-        String host;
-        int port = 1099;
-        switch (args.length) {
-            case 3:
-                port = Integer.parseInt(args[2]);
-            case 2:
-                host = args[1];
-                break;
-            default:
-                System.err.println("One or Two arguments expected: <cchost> [<ccport>]");
-                return;
-        }
-        IHyracksClientConnection hcc = new HyracksRMIConnection(host, port);
+    private static class Options {
+        @Option(name = "-host", usage = "Hyracks Cluster Controller Host name", required = true)
+        public String host;
 
-        JobSpecification job = createJob();
+        @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)", required = false)
+        public int port = 1099;
+
+        @Option(name = "-app", usage = "Hyracks Application name", required = true)
+        public String app;
+
+        @Option(name = "-infile-customer-splits", usage = "Comma separated list of file-splits for the CUSTOMER input. A file-split is <node-name>:<path>", required = true)
+        public String inFileCustomerSplits;
+
+        @Option(name = "-infile-order-splits", usage = "Comma separated list of file-splits for the ORDER input. A file-split is <node-name>:<path>", required = true)
+        public String inFileOrderSplits;
+
+        @Option(name = "-outfile-splits", usage = "Comma separated list of file-splits for the output", required = true)
+        public String outFileSplits;
+
+        @Option(name = "-num-join-partitions", usage = "Number of Join partitions to use (default: 1)", required = false)
+        public int numJoinPartitions = 1;
+
+        @Option(name = "-profile", usage = "Enable/Disable profiling. (default: enabled)")
+        public boolean profile = true;
+    }
+
+    public static void main(String[] args) throws Exception {
+        Options options = new Options();
+        CmdLineParser parser = new CmdLineParser(options);
+        parser.parseArgument(args);
+
+        IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
+
+        JobSpecification job = createJob(parseFileSplits(options.inFileCustomerSplits),
+                parseFileSplits(options.inFileOrderSplits), parseFileSplits(options.outFileSplits),
+                options.numJoinPartitions);
 
         long start = System.currentTimeMillis();
-        UUID jobId = hcc.createJob(appName, job);
+        UUID jobId = hcc.createJob(options.app, job,
+                options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
         hcc.start(jobId);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
         System.err.println(start + " " + end + " " + (end - start));
     }
 
-    private static JobSpecification createJob() {
+    private static FileSplit[] parseFileSplits(String fileSplits) {
+        String[] splits = fileSplits.split(",");
+        FileSplit[] fSplits = new FileSplit[splits.length];
+        for (int i = 0; i < splits.length; ++i) {
+            String s = splits[i].trim();
+            int idx = s.indexOf(':');
+            if (idx < 0) {
+                throw new IllegalArgumentException("File split " + s + " not well formed");
+            }
+            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
+        }
+        return fSplits;
+    }
+
+    private static JobSpecification createJob(FileSplit[] customerSplits, FileSplit[] orderSplits,
+            FileSplit[] resultSplits, int numJoinPartitions) {
         JobSpecification spec = new JobSpecification();
 
-        FileSplit[] custSplits = createCustomerFileSplits();
-        IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
+        IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(customerSplits);
         RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
-        FileSplit[] ordersSplits = createOrdersFileSplits();
-        IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
+        IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(orderSplits);
         RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
                 UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
@@ -103,7 +151,7 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        ordScanner.setPartitionConstraint(createRRPartitionConstraint(2));
+        createPartitionConstraint(spec, ordScanner, orderSplits);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
                 new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
@@ -111,13 +159,13 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
-        custScanner.setPartitionConstraint(createRRPartitionConstraint(2));
+        createPartitionConstraint(spec, custScanner, customerSplits);
 
         InMemoryHashJoinOperatorDescriptor join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 0 },
                 new int[] { 1 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE },
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, custOrderJoinDesc,
                 6000000);
-        join.setPartitionConstraint(new PartitionCountConstraint(4));
+        PartitionConstraintHelper.addPartitionCountConstraint(spec, join, numJoinPartitions);
 
         RecordDescriptor groupResultDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
@@ -130,10 +178,11 @@
                 new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
                 new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
                 groupResultDesc, 16);
-        gby.setPartitionConstraint(new PartitionCountConstraint(4));
+        createPartitionConstraint(spec, gby, resultSplits);
 
-        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        printer.setPartitionConstraint(new PartitionCountConstraint(4));
+        IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(resultSplits);
+        FrameFileWriterOperatorDescriptor writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
+        createPartitionConstraint(spec, writer, resultSplits);
 
         IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
@@ -151,45 +200,17 @@
         spec.connect(joinGroupConn, join, 0, gby, 0);
 
         IConnectorDescriptor gbyPrinterConn = new OneToOneConnectorDescriptor(spec);
-        spec.connect(gbyPrinterConn, gby, 0, printer, 0);
+        spec.connect(gbyPrinterConn, gby, 0, writer, 0);
 
-        spec.addRoot(printer);
+        spec.addRoot(writer);
         return spec;
     }
 
-    private static FileSplit[] createOrdersFileSplits() {
-        FileSplit fss[] = new FileSplit[2];
-        for (int i = 0; i < fss.length; ++i) {
-            fss[i] = new FileSplit("foo", new FileReference(new File("data/tpch0.001/orders-part" + (i + 1) + ".tbl")));
+    private static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
+        String[] parts = new String[splits.length];
+        for (int i = 0; i < splits.length; ++i) {
+            parts[i] = splits[i].getNodeName();
         }
-        return fss;
-    }
-
-    private static FileSplit[] createCustomerFileSplits() {
-        FileSplit fss[] = new FileSplit[2];
-        for (int i = 0; i < fss.length; ++i) {
-            fss[i] = new FileSplit("foo",
-                    new FileReference(new File("data/tpch0.001/customer-part" + (i + 1) + ".tbl")));
-        }
-        return fss;
-    }
-
-    private static final LocationConstraint[] LCS = { new AbsoluteLocationConstraint("NC1"),
-            new AbsoluteLocationConstraint("NC2") };
-
-    private static PartitionConstraint createRRPartitionConstraint(int k) {
-        LocationConstraint[] lcs = new LocationConstraint[2];
-        for (int i = 0; i < lcs.length; ++i) {
-            lcs[i] = createRRSteppedChoiceConstraint(i, k);
-        }
-        return new ExplicitPartitionConstraint(lcs);
-    }
-
-    private static LocationConstraint createRRSteppedChoiceConstraint(int index, int choices) {
-        LocationConstraint[] lcs = new LocationConstraint[choices];
-        for (int i = 0; i < choices; ++i) {
-            lcs[i] = LCS[(index + i) % LCS.length];
-        }
-        return new ChoiceLocationConstraint(lcs);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, parts);
     }
 }
\ No newline at end of file
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 7be5938..d0df7f1 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
@@ -4,6 +4,8 @@
 import java.net.InetSocketAddress;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@@ -20,11 +22,10 @@
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.util.ReflectionUtils;
 
-import edu.uci.ics.hyracks.api.constraints.ExplicitPartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraint;
-import edu.uci.ics.hyracks.api.constraints.PartitionCountConstraint;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 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.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
@@ -48,6 +49,7 @@
 
     public static final String FS_DEFAULT_NAME = "fs.default.name";
     private JobConf jobConf;
+    private Map<OperatorDescriptorId,Integer> operatorInstanceCount = new HashMap<OperatorDescriptorId,Integer>();
     public static final String HYRACKS_EX_SORT_FRAME_LIMIT = "HYRACKS_EX_SORT_FRAME_LIMIT"; 
     public static final int DEFAULT_EX_SORT_FRAME_LIMIT = 4096;
     public static final int DEFAULT_MAX_MAPPERS = 40;
@@ -130,22 +132,26 @@
       	InputFormat inputFormat = conf.getInputFormat();
     	return inputFormat.getSplits(conf, conf.getNumMapTasks());
     }
-    
+   
+    private void configurePartitionCountConstraint(JobSpecification spec, IOperatorDescriptor operator,int instanceCount){
+        PartitionConstraintHelper.addPartitionCountConstraint(spec, operator, instanceCount);
+        operatorInstanceCount.put(operator.getOperatorId(),instanceCount);
+    }
+
     public HadoopMapperOperatorDescriptor getMapper(JobConf conf,JobSpecification spec, IOperatorDescriptor previousOp)
             throws Exception {
         boolean selfRead = previousOp == null;
         IHadoopClassFactory classFactory = new ClasspathBasedHadoopClassFactory();
         HadoopMapperOperatorDescriptor mapOp = null;
-        PartitionConstraint constraint;
         if(selfRead) {
             Object [] splits = getInputSplits(conf,maxMappers);
             mapOp = new HadoopMapperOperatorDescriptor(spec, conf, splits,classFactory);
-            mapOp.setPartitionConstraint(new PartitionCountConstraint(splits.length));
+	    configurePartitionCountConstraint(spec,mapOp,splits.length);
             System.out.println("No of  mappers :" + splits.length);
         } else {
-            constraint = previousOp.getPartitionConstraint();
-            mapOp.setPartitionConstraint(constraint);
+	    configurePartitionCountConstraint(spec,mapOp,getInstanceCount(previousOp));
             mapOp = new HadoopMapperOperatorDescriptor(spec,conf,classFactory);
+            spec.connect(new OneToOneConnectorDescriptor(spec), previousOp, 0, mapOp, 0);
         }
         return mapOp;
     }
@@ -176,50 +182,55 @@
         return spec;
     }
 
-    private IOperatorDescriptor configureOutput(IOperatorDescriptor previousOperator, JobConf conf,
+    private IOperatorDescriptor configureOutput( IOperatorDescriptor previousOperator, JobConf conf,
             JobSpecification spec) throws Exception {
-        PartitionConstraint previousOpConstraint = previousOperator.getPartitionConstraint();
-        int noOfInputs = previousOpConstraint instanceof PartitionCountConstraint ? ((PartitionCountConstraint) previousOpConstraint)
-                .getCount()
-                : ((ExplicitPartitionConstraint) previousOpConstraint).getLocationConstraints().length;
-        int numOutputters = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : noOfInputs;
+	int instanceCountPreviousOperator = operatorInstanceCount.get(previousOperator.getOperatorId());
+        int numOutputters = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : instanceCountPreviousOperator;
         HadoopWriteOperatorDescriptor writer = null;
         writer = new HadoopWriteOperatorDescriptor(spec, conf, numOutputters);
-        writer.setPartitionConstraint(previousOperator.getPartitionConstraint());
+	configurePartitionCountConstraint(spec,writer,numOutputters);
         spec.connect(new OneToOneConnectorDescriptor(spec), previousOperator, 0, writer, 0);
         return writer;
     }
 
+
+    private int getInstanceCount(IOperatorDescriptor operator) {
+        return operatorInstanceCount.get(operator.getOperatorId());
+    } 
+
     private IOperatorDescriptor addCombiner(IOperatorDescriptor previousOperator, JobConf jobConf,
             JobSpecification spec) throws Exception {
         boolean useCombiner = (jobConf.getCombinerClass() != null);
         IOperatorDescriptor mapSideOutputOp = previousOperator;
         if (useCombiner) {
             System.out.println("Using Combiner:" + jobConf.getCombinerClass().getName());
-            PartitionConstraint mapperPartitionConstraint = previousOperator.getPartitionConstraint();
             IOperatorDescriptor mapSideCombineSortOp = getExternalSorter(jobConf, spec);
-            mapSideCombineSortOp.setPartitionConstraint(mapperPartitionConstraint);
+	    configurePartitionCountConstraint(spec,mapSideCombineSortOp,getInstanceCount(previousOperator));
     
             HadoopReducerOperatorDescriptor mapSideCombineReduceOp = getReducer(jobConf, spec);
-            mapSideCombineReduceOp.setPartitionConstraint(mapperPartitionConstraint);
-                  spec.connect(new OneToOneConnectorDescriptor(spec), previousOperator, 0, mapSideCombineSortOp, 0);
+	    configurePartitionCountConstraint(spec,mapSideCombineReduceOp,getInstanceCount(previousOperator));
+            spec.connect(new OneToOneConnectorDescriptor(spec), previousOperator, 0, mapSideCombineSortOp, 0);
             spec.connect(new OneToOneConnectorDescriptor(spec), mapSideCombineSortOp, 0, mapSideCombineReduceOp, 0);
             mapSideOutputOp = mapSideCombineSortOp;
         }
         return mapSideOutputOp;
     }
     
+    private int getNumReduceTasks(JobConf jobConf) {
+        int numReduceTasks = Math.min(maxReducers,jobConf.getNumReduceTasks());
+        return numReduceTasks;
+    }
+    
     private IOperatorDescriptor addReducer(IOperatorDescriptor previousOperator, JobConf jobConf,
             JobSpecification spec) throws Exception {
         IOperatorDescriptor mrOutputOperator = previousOperator;
         if (jobConf.getNumReduceTasks() != 0) {
             IOperatorDescriptor sorter = getExternalSorter(jobConf, spec);
             HadoopReducerOperatorDescriptor reducer = getReducer(jobConf, spec);
-            int numReduceTasks = Math.min(maxReducers,jobConf.getNumReduceTasks());
+            int numReduceTasks = getNumReduceTasks(jobConf);
             System.out.println("No of Reducers :" + numReduceTasks);
-            PartitionConstraint reducerPartitionConstraint = new PartitionCountConstraint(numReduceTasks);
-            sorter.setPartitionConstraint(reducerPartitionConstraint);
-            reducer.setPartitionConstraint(reducerPartitionConstraint);
+	    configurePartitionCountConstraint(spec,sorter,numReduceTasks);
+	    configurePartitionCountConstraint(spec,reducer,numReduceTasks);
     
             IConnectorDescriptor mToNConnectorDescriptor = getMtoNHashPartitioningConnector(jobConf, spec);
             spec.connect(mToNConnectorDescriptor, previousOperator, 0, sorter, 0);
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/job/profiling/CounterContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/CounterContext.java
similarity index 91%
rename from hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/job/profiling/CounterContext.java
rename to hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/CounterContext.java
index 401dd57..689a524 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/job/profiling/CounterContext.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/CounterContext.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.nc.job.profiling;
+package edu.uci.ics.hyracks.test.support;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -40,11 +40,6 @@
         return counter;
     }
 
-    @Override
-    public String getContextName() {
-        return contextName;
-    }
-
     public synchronized void dump(Map<String, Long> dumpMap) {
         for (Counter c : counterMap.values()) {
             dumpMap.put(c.getName(), c.get());
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 921cb55..2fc43fc 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
@@ -27,7 +27,6 @@
 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.job.profiling.CounterContext;
 
 public class TestJobletContext implements IHyracksJobletContext {
     private final INCApplicationContext appContext;
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
index d28911b..4d31326 100644
--- 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
@@ -27,7 +27,6 @@
 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.job.profiling.CounterContext;
 
 public class TestStageletContext implements IHyracksStageletContext {
     private final IHyracksJobletContext jobletContext;
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/.classpath b/hyracks-tests/hyracks-storage-am-btree-test/.classpath
index e44aa2f..f2cc5f7 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/.classpath
+++ b/hyracks-tests/hyracks-storage-am-btree-test/.classpath
@@ -1,6 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <classpath>
-	<classpathentry kind="src" path="src/test/java"/>
+	<classpathentry kind="src" output="target/test-classes" path="src/test/java"/>
 	<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-tests/hyracks-storage-am-invertedindex-test/.classpath b/hyracks-tests/hyracks-storage-am-invertedindex-test/.classpath
index e44aa2f..f2cc5f7 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/.classpath
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/.classpath
@@ -1,6 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <classpath>
-	<classpathentry kind="src" path="src/test/java"/>
+	<classpathentry kind="src" output="target/test-classes" path="src/test/java"/>
 	<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/pom.xml b/pom.xml
index 0c31f34..684e8b7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -60,14 +60,14 @@
     <module>hyracks-control-cc</module>
     <module>hyracks-control-nc</module>
     <module>hyracks-cli</module>
-    <module>hyracks-hadoop-compat</module>
-    <module>hyracks-server</module>
     <module>hyracks-storage-common</module>
     <module>hyracks-storage-am-btree</module>
     <module>hyracks-storage-am-invertedindex</module>
     <module>hyracks-test-support</module>
     <module>hyracks-tests</module>
+    <module>hyracks-server</module>
     <module>hyracks-examples</module>
     <module>hyracks-documentation</module>
+    <module>hyracks-hadoop-compat</module>
   </modules>
 </project>