merged from fullstack_asterix_stabilization to fullstack_lsm_staging -r3100:3171

git-svn-id: https://hyracks.googlecode.com/svn/branches/fullstack_lsm_staging@3173 123451ca-8445-de46-9d55-352943316053
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
index 5234d2c..b8bdf3e 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
@@ -20,6 +20,7 @@
     CLUSTER,
     DATASOURCESCAN,
     DISTINCT,
+    DISTRIBUTE_RESULT,
     GROUP,
     EMPTYTUPLESOURCE,
     EXCHANGE,
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index 0845d05..0efb5ff 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -7,6 +7,7 @@
     BTREE_SEARCH,
     STATS,
     DATASOURCE_SCAN,
+    DISTRIBUTE_RESULT,
     EMPTY_TUPLE_SOURCE,
     EXTERNAL_GROUP_BY,
     IN_MEMORY_HASH_JOIN,
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index baf8822..d516d57 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -51,6 +51,10 @@
             int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc)
             throws AlgebricksException;
 
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
+            int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc, boolean ordered,
+            JobSpecification spec) throws AlgebricksException;
+
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(IDataSource<S> dataSource,
             IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
             JobGenContext context, JobSpecification jobSpec) throws AlgebricksException;
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistributeResultOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistributeResultOperator.java
new file mode 100644
index 0000000..6ca6d87
--- /dev/null
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistributeResultOperator.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+public class DistributeResultOperator extends AbstractLogicalOperator {
+    private List<Mutable<ILogicalExpression>> expressions;
+    private IDataSink dataSink;
+
+    public DistributeResultOperator(List<Mutable<ILogicalExpression>> expressions, IDataSink dataSink) {
+        this.expressions = expressions;
+        this.dataSink = dataSink;
+    }
+
+    public List<Mutable<ILogicalExpression>> getExpressions() {
+        return expressions;
+    }
+
+    public IDataSink getDataSink() {
+        return dataSink;
+    }
+
+    @Override
+    public LogicalOperatorTag getOperatorTag() {
+        return LogicalOperatorTag.DISTRIBUTE_RESULT;
+    }
+
+    @Override
+    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+        return visitor.visitDistributeResultOperator(this, arg);
+    }
+
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+        boolean modif = false;
+        for (int i = 0; i < expressions.size(); i++) {
+            boolean b = visitor.transform(expressions.get(i));
+            if (b) {
+                modif = true;
+            }
+        }
+        return modif;
+    }
+
+    @Override
+    public VariablePropagationPolicy getVariablePropagationPolicy() {
+        return VariablePropagationPolicy.ALL;
+    }
+
+    @Override
+    public boolean isMap() {
+        return false; // actually depends on the physical op.
+    }
+
+    @Override
+    public void recomputeSchema() {
+        schema = new ArrayList<LogicalVariable>();
+        schema.addAll(inputs.get(0).getValue().getSchema());
+    }
+
+    @Override
+    public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+        return createPropagatingAllInputsTypeEnvironment(ctx);
+    }
+
+}
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index 0539cbe..1b4be1e 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -47,8 +47,10 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -64,7 +66,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -476,6 +477,14 @@
     }
 
     @Override
+    public Void visitDistributeResultOperator(DistributeResultOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        // propagateFDsAndEquivClasses(op, ctx);
+        setEmptyFDsEqClasses(op, ctx);
+        return null;
+    }
+
+    @Override
     public Void visitWriteResultOperator(WriteResultOperator op, IOptimizationContext ctx) throws AlgebricksException {
         // propagateFDsAndEquivClasses(op, ctx);
         setEmptyFDsEqClasses(op, ctx);
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index b97597d..ac6d887 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -38,6 +38,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
@@ -425,6 +426,17 @@
     }
 
     @Override
+    public Boolean visitDistributeResultOperator(DistributeResultOperator op, ILogicalOperator arg)
+            throws AlgebricksException {
+        AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+        if (aop.getOperatorTag() != LogicalOperatorTag.DISTRIBUTE_RESULT)
+            return Boolean.FALSE;
+        DistributeResultOperator writeOpArg = (DistributeResultOperator) copyAndSubstituteVar(op, arg);
+        boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getSchema(), writeOpArg.getSchema());
+        return isomorphic;
+    }
+
+    @Override
     public Boolean visitWriteResultOperator(WriteResultOperator op, ILogicalOperator arg) throws AlgebricksException {
         AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
         if (aop.getOperatorTag() != LogicalOperatorTag.WRITE_RESULT)
@@ -762,6 +774,14 @@
         }
 
         @Override
+        public ILogicalOperator visitDistributeResultOperator(DistributeResultOperator op, Void arg)
+                throws AlgebricksException {
+            ArrayList<Mutable<ILogicalExpression>> newExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+            deepCopyExpressionRefs(newExpressions, op.getExpressions());
+            return new DistributeResultOperator(newExpressions, op.getDataSink());
+        }
+
+        @Override
         public ILogicalOperator visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {
             ArrayList<Mutable<ILogicalExpression>> newKeyExpressions = new ArrayList<Mutable<ILogicalExpression>>();
             deepCopyExpressionRefs(newKeyExpressions, op.getKeyExpressions());
@@ -784,8 +804,8 @@
             deepCopyExpressionRefs(newPrimaryKeyExpressions, op.getPrimaryKeyExpressions());
             List<Mutable<ILogicalExpression>> newSecondaryKeyExpressions = new ArrayList<Mutable<ILogicalExpression>>();
             deepCopyExpressionRefs(newSecondaryKeyExpressions, op.getSecondaryKeyExpressions());
-            Mutable<ILogicalExpression> newFilterExpression = new MutableObject<ILogicalExpression>(((AbstractLogicalExpression)op.getFilterExpression())
-                    .cloneExpression());
+            Mutable<ILogicalExpression> newFilterExpression = new MutableObject<ILogicalExpression>(
+                    ((AbstractLogicalExpression) op.getFilterExpression()).cloneExpression());
             return new IndexInsertDeleteOperator(op.getDataSourceIndex(), newPrimaryKeyExpressions,
                     newSecondaryKeyExpressions, newFilterExpression, op.getOperation());
         }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index 562bb4c..b9544c7 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -37,8 +37,10 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -54,7 +56,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -230,6 +231,13 @@
     }
 
     @Override
+    public Void visitDistributeResultOperator(DistributeResultOperator op, ILogicalOperator arg)
+            throws AlgebricksException {
+        mapVariablesStandard(op, arg);
+        return null;
+    }
+
+    @Override
     public Void visitWriteResultOperator(WriteResultOperator op, ILogicalOperator arg) throws AlgebricksException {
         mapVariablesStandard(op, arg);
         return null;
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index 9b2f5a0..8f1d686 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -29,8 +29,10 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -46,7 +48,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -240,6 +241,12 @@
     }
 
     @Override
+    public Void visitDistributeResultOperator(DistributeResultOperator op, IOptimizationContext arg)
+            throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Void visitWriteResultOperator(WriteResultOperator op, IOptimizationContext arg) throws AlgebricksException {
         // TODO Auto-generated method stub
         return null;
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
index 78b6801..31adcba 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
@@ -32,8 +32,10 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;

+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;

+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;

@@ -49,7 +51,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;

@@ -223,6 +224,11 @@
     }

 

     @Override

+    public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) throws AlgebricksException {

+        return null;

+    }

+

+    @Override

     public Void visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {

         return null;

     }

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index a759e35..cd0cee3 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -31,8 +31,10 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;

+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;

+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;

@@ -48,7 +50,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;

@@ -238,6 +239,12 @@
     }

 

     @Override

+    public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) throws AlgebricksException {

+        standardLayout(op);

+        return null;

+    }

+

+    @Override

     public Void visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {

         standardLayout(op);

         return null;

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index 11e56ca..69fb3f8 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -33,8 +33,10 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;

+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;

+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;

@@ -51,7 +53,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;

-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;

@@ -335,6 +336,16 @@
     }

 

     @Override

+    public Void visitDistributeResultOperator(DistributeResultOperator op, Pair<LogicalVariable, LogicalVariable> pair)

+            throws AlgebricksException {

+        for (Mutable<ILogicalExpression> e : op.getExpressions()) {

+            e.getValue().substituteVar(pair.first, pair.second);

+        }

+        substVarTypes(op, pair);

+        return null;

+    }

+

+    @Override

     public Void visitWriteResultOperator(WriteResultOperator op, Pair<LogicalVariable, LogicalVariable> pair)

             throws AlgebricksException {

         op.getPayloadExpression().getValue().substituteVar(pair.first, pair.second);

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index 0ea9367..5361a19 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -32,6 +32,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;

+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;

 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;

@@ -304,6 +305,14 @@
     }

 

     @Override

+    public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) {

+        for (Mutable<ILogicalExpression> expr : op.getExpressions()) {

+            expr.getValue().getUsedVariables(usedVariables);

+        }

+        return null;

+    }

+

+    @Override

     public Void visitWriteResultOperator(WriteResultOperator op, Void arg) {

         op.getPayloadExpression().getValue().getUsedVariables(usedVariables);

         for (Mutable<ILogicalExpression> e : op.getKeyExpressions()) {

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
new file mode 100644
index 0000000..302d4d2
--- /dev/null
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.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.algebricks.core.algebra.operators.physical;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class DistributeResultPOperator extends AbstractPhysicalOperator {
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.DISTRIBUTE_RESULT;
+    }
+
+    @Override
+    public boolean isMicroOperator() {
+        return false;
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+        ILogicalOperator op2 = op.getInputs().get(0).getValue();
+        deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+    }
+
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector reqdByParent) {
+        DistributeResultOperator write = (DistributeResultOperator) op;
+        IDataSink sink = write.getDataSink();
+        IPartitioningProperty pp = sink.getPartitioningProperty();
+        StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(pp, null) };
+        return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+    }
+
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    @Override
+    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+            IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+            throws AlgebricksException {
+        DistributeResultOperator resultOp = (DistributeResultOperator) op;
+        IMetadataProvider mp = context.getMetadataProvider();
+
+        JobSpecification spec = builder.getJobSpec();
+
+        int[] columns = new int[resultOp.getExpressions().size()];
+        int i = 0;
+        for (Mutable<ILogicalExpression> exprRef : resultOp.getExpressions()) {
+            ILogicalExpression expr = exprRef.getValue();
+            if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                throw new NotImplementedException("Only writing variable expressions is supported.");
+            }
+            VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
+            LogicalVariable v = varRef.getVariableReference();
+            columns[i++] = inputSchemas[0].findVariable(v);
+        }
+        RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(
+                context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
+
+        IPrinterFactory[] pf = JobGenHelper.mkPrinterFactories(inputSchemas[0], context.getTypeEnvironment(op),
+                context, columns);
+
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getResultHandleRuntime(
+                resultOp.getDataSink(), columns, pf, inputDesc, false, spec);
+
+        builder.contributeHyracksOperator(resultOp, runtimeAndConstraints.first);
+        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+        ILogicalOperator src = resultOp.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(src, 0, resultOp, 0);
+    }
+}
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index e22a6b7..fc0c433 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
@@ -164,6 +165,13 @@
     }
 
     @Override
+    public String visitDistributeResultOperator(DistributeResultOperator op, Integer indent) {
+        StringBuilder buffer = new StringBuilder();
+        addIndent(buffer, indent).append("distribute result ").append(op.getExpressions());
+        return buffer.toString();
+    }
+
+    @Override
     public String visitWriteResultOperator(WriteResultOperator op, Integer indent) {
         StringBuilder buffer = new StringBuilder();
         addIndent(buffer, indent).append("load ").append(op.getDataSource()).append(" from ")
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
index 6b5949e..23dac2a 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
@@ -20,8 +20,10 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -37,7 +39,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -97,6 +98,8 @@
 
     public R visitWriteOperator(WriteOperator op, T arg) throws AlgebricksException;
 
+    public R visitDistributeResultOperator(DistributeResultOperator op, T arg) throws AlgebricksException;
+
     public R visitWriteResultOperator(WriteResultOperator op, T arg) throws AlgebricksException;
 
     public R visitInsertDeleteOperator(InsertDeleteOperator op, T tag) throws AlgebricksException;
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/config/AlgebricksConfig.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/config/AlgebricksConfig.java
index 11ec043..061dbe4 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -20,20 +20,4 @@
     public static final boolean DEBUG = true;
     public static final String ALGEBRICKS_LOGGER_NAME = "edu.uci.ics.hyracks.algebricks";
     public static final Logger ALGEBRICKS_LOGGER = Logger.getLogger(ALGEBRICKS_LOGGER_NAME);
-    public static final String HYRACKS_APP_NAME = "algebricks";
-
-    // public static final Level ALGEBRICKS_LOG_LEVEL = Level.FINEST;
-    //
-    // static {
-    // Handler h;
-    // try {
-    // h = new ConsoleHandler();
-    // h.setFormatter(new SysoutFormatter());
-    // } catch (Exception e) {
-    // h = new ConsoleHandler();
-    // }
-    // h.setLevel(ALGEBRICKS_LOG_LEVEL);
-    // ALGEBRICKS_LOGGER.addHandler(h);
-    // ALGEBRICKS_LOGGER.setLevel(ALGEBRICKS_LOG_LEVEL);
-    // }
 }
diff --git a/algebricks/algebricks-data/src/main/java/edu/uci/ics/hyracks/algebricks/data/IResultSerializerFactoryProvider.java b/algebricks/algebricks-data/src/main/java/edu/uci/ics/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
new file mode 100644
index 0000000..5e62612
--- /dev/null
+++ b/algebricks/algebricks-data/src/main/java/edu/uci/ics/hyracks/algebricks/data/IResultSerializerFactoryProvider.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.algebricks.data;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializerFactory;
+
+public interface IResultSerializerFactoryProvider extends Serializable {
+    /**
+     * Returns a result serializer factory
+     * 
+     * @param fields
+     *            - A position of the fields in the order it should be written in the output.
+     * @param printerFactories
+     *            - A printer factory array to print the tuple containing different fields.
+     * @param writerFactory
+     *            - A writer factory to write the serialized data to the print stream.
+     * @param inputRecordDesc
+     *            - The record descriptor describing the input frame to be serialized.
+     * @return A new instance of result serialized appender.
+     */
+    public IResultSerializerFactory getAqlResultSerializerFactoryProvider(int[] fields,
+            IPrinterFactory[] printerFactories, IAWriterFactory writerFactory);
+}
diff --git a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
index 740450e..6c2700d 100644
--- a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
+++ b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
@@ -127,9 +127,9 @@
             }
         });
         builder.setTypeTraitProvider(new ITypeTraitProvider() {
-			public ITypeTraits getTypeTrait(Object type) {
-				return null;
-			}
+            public ITypeTraits getTypeTrait(Object type) {
+                return null;
+            }
         });
         builder.setPrinterProvider(PigletPrinterFactoryProvider.INSTANCE);
         builder.setExpressionRuntimeProvider(new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter(
diff --git a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
index f1959e2..498b43a 100644
--- a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
+++ b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
@@ -142,6 +142,13 @@
     }
 
     @Override
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
+            int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc, boolean ordered,
+            JobSpecification spec) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(
             IDataSource<String> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
             LogicalVariable payLoadVar, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException {
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index 38cf96e..60a4fbb 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -38,6 +38,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AggregatePOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.DataSourceScanPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.DistributeResultPOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.EmptyTupleSourcePOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.InMemoryStableSortPOperator;
@@ -245,6 +246,10 @@
                     op.setPhysicalOperator(new SinkWritePOperator());
                     break;
                 }
+                case DISTRIBUTE_RESULT: {
+                    op.setPhysicalOperator(new DistributeResultPOperator());
+                    break;
+                }
                 case WRITE_RESULT: {
                     WriteResultOperator opLoad = (WriteResultOperator) op;
                     LogicalVariable payload;
@@ -267,8 +272,8 @@
                     List<LogicalVariable> secondaryKeys = new ArrayList<LogicalVariable>();
                     getKeys(opInsDel.getPrimaryKeyExpressions(), primaryKeys);
                     getKeys(opInsDel.getSecondaryKeyExpressions(), secondaryKeys);
-                    op.setPhysicalOperator(new IndexInsertDeletePOperator(primaryKeys, secondaryKeys, 
-                    		opInsDel.getFilterExpression(), opInsDel.getDataSourceIndex()));
+                    op.setPhysicalOperator(new IndexInsertDeletePOperator(primaryKeys, secondaryKeys, opInsDel
+                            .getFilterExpression(), opInsDel.getDataSourceIndex()));
                     break;
                 }
                 case SINK: {
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
new file mode 100644
index 0000000..4f28c81
--- /dev/null
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.algebricks.runtime.serializer;
+
+import java.io.PrintStream;
+import java.nio.BufferOverflowException;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.IAWriter;
+import edu.uci.ics.hyracks.algebricks.data.IAWriterFactory;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.data.IResultSerializerFactoryProvider;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ResultSerializerFactoryProvider implements IResultSerializerFactoryProvider {
+    private static final long serialVersionUID = 1L;
+
+    public static final ResultSerializerFactoryProvider INSTANCE = new ResultSerializerFactoryProvider();
+
+    private ResultSerializerFactoryProvider() {
+    }
+
+    @Override
+    public IResultSerializerFactory getAqlResultSerializerFactoryProvider(final int[] fields,
+            final IPrinterFactory[] printerFactories, final IAWriterFactory writerFactory) {
+        return new IResultSerializerFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IResultSerializer createResultSerializer(RecordDescriptor inputRecordDesc, PrintStream printStream) {
+                final IAWriter writer = writerFactory.createWriter(fields, printStream, printerFactories,
+                        inputRecordDesc);
+
+                return new IResultSerializer() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public void init() throws HyracksDataException {
+                        try {
+                            writer.init();
+                        } catch (AlgebricksException e) {
+                            throw new HyracksDataException(e);
+                        }
+                    }
+
+                    @Override
+                    public boolean appendTuple(IFrameTupleAccessor tAccess, int tIdx) throws HyracksDataException {
+                        try {
+                            writer.printTuple(tAccess, tIdx);
+                        } catch (BufferOverflowException e) {
+                            return false;
+                        } catch (AlgebricksException e) {
+                            throw new HyracksDataException(e);
+                        }
+                        return true;
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/algebricks/algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java b/algebricks/algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
index b5c4b47..2855eb2 100644
--- a/algebricks/algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
+++ b/algebricks/algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
@@ -3,9 +3,9 @@
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * you may obtain a copy of the License from
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -55,6 +55,7 @@
         ncConfig1.ccPort = TEST_HYRACKS_CC_CLUSTER_NET_PORT;
         ncConfig1.clusterNetIPAddress = "127.0.0.1";
         ncConfig1.dataIPAddress = "127.0.0.1";
+        ncConfig1.datasetIPAddress = "127.0.0.1";
         ncConfig1.nodeId = NC1_ID;
         nc1 = new NodeControllerService(ncConfig1);
         nc1.start();
@@ -64,12 +65,12 @@
         ncConfig2.ccPort = TEST_HYRACKS_CC_CLUSTER_NET_PORT;
         ncConfig2.clusterNetIPAddress = "127.0.0.1";
         ncConfig2.dataIPAddress = "127.0.0.1";
+        ncConfig2.datasetIPAddress = "127.0.0.1";
         ncConfig2.nodeId = NC2_ID;
         nc2 = new NodeControllerService(ncConfig2);
         nc2.start();
 
         hcc = new HyracksConnection(ccConfig.clientNetIpAddress, ccConfig.clientNetPort);
-        hcc.createApplication(AlgebricksConfig.HYRACKS_APP_NAME, null);
     }
 
     public static void deinit() throws Exception {
@@ -80,9 +81,9 @@
 
     public static void runJob(JobSpecification spec) throws Exception {
         AlgebricksConfig.ALGEBRICKS_LOGGER.info(spec.toJSON().toString());
-        JobId jobId = hcc.startJob(AlgebricksConfig.HYRACKS_APP_NAME, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+        JobId jobId = hcc.startJob(spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
         AlgebricksConfig.ALGEBRICKS_LOGGER.info(jobId.toString());
         hcc.waitForCompletion(jobId);
     }
 
-}
\ No newline at end of file
+}
diff --git a/hivesterix/hivesterix-common/src/main/java/edu/uci/ics/hivesterix/common/config/ConfUtil.java b/hivesterix/hivesterix-common/src/main/java/edu/uci/ics/hivesterix/common/config/ConfUtil.java
index 666d361..648deb6 100644
--- a/hivesterix/hivesterix-common/src/main/java/edu/uci/ics/hivesterix/common/config/ConfUtil.java
+++ b/hivesterix/hivesterix-common/src/main/java/edu/uci/ics/hivesterix/common/config/ConfUtil.java
@@ -1,7 +1,6 @@
 package edu.uci.ics.hivesterix.common.config;

 

 import java.io.BufferedReader;

-import java.io.DataInputStream;

 import java.io.FileInputStream;

 import java.io.InputStream;

 import java.io.InputStreamReader;

@@ -28,6 +27,7 @@
 @SuppressWarnings({ "rawtypes", "deprecation" })

 public class ConfUtil {

     private static final String clusterPropertiesPath = "conf/cluster.properties";

+    private static final String masterFilePath = "conf/master";

 

     private static JobConf job;

     private static HiveConf hconf;

@@ -134,15 +134,27 @@
                 clusterProps.load(confIn);

                 confIn.close();

             }

-            Process process = Runtime.getRuntime().exec("src/main/resources/scripts/getip.sh");

-            BufferedReader ipReader = new BufferedReader(new InputStreamReader(new DataInputStream(

-                    process.getInputStream())));

-            String ipAddress = ipReader.readLine();

-            ipReader.close();

-            int port = Integer.parseInt(clusterProps.getProperty("CC_CLIENTPORT"));

-            int mpl = Integer.parseInt(hconf.get("hive.hyracks.parrallelism"));

 

-            hcc = new HyracksConnection(ipAddress, port);

+            if (hcc == null) {

+                BufferedReader ipReader = new BufferedReader(new InputStreamReader(new FileInputStream(masterFilePath)));

+                String masterNode = ipReader.readLine();

+                ipReader.close();

+

+                InetAddress[] ips = InetAddress.getAllByName(masterNode);

+                int port = Integer.parseInt(clusterProps.getProperty("CC_CLIENTPORT"));

+                for (InetAddress ip : ips) {

+                    if (ip.getAddress().length <= 4) {

+                        try {

+                            hcc = new HyracksConnection(ip.getHostAddress(), port);

+                            break;

+                        } catch (Exception e) {

+                            continue;

+                        }

+                    }

+                }

+            }

+

+            int mpl = Integer.parseInt(hconf.get("hive.hyracks.parrallelism"));

             topology = hcc.getClusterTopology();

             ncNameToNcInfos = hcc.getNodeControllerInfos();

             NCs = new String[ncNameToNcInfos.size() * mpl];

diff --git a/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java b/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
index c39485f..53c9d42 100644
--- a/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
+++ b/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
@@ -1,12 +1,12 @@
 package edu.uci.ics.hivesterix.runtime.exec;

 

 import java.io.BufferedReader;

-import java.io.DataInputStream;

 import java.io.FileInputStream;

 import java.io.InputStream;

 import java.io.InputStreamReader;

 import java.io.PrintWriter;

 import java.io.Serializable;

+import java.net.InetAddress;

 import java.util.ArrayList;

 import java.util.HashMap;

 import java.util.Iterator;

@@ -80,6 +80,7 @@
 

     private static final Log LOG = LogFactory.getLog(HyracksExecutionEngine.class.getName());

     private static final String clusterPropertiesPath = "conf/cluster.properties";

+    private static final String masterFilePath = "conf/master";

 

     private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> DEFAULT_LOGICAL_REWRITES = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();

     private static List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> DEFAULT_PHYSICAL_REWRITES = new ArrayList<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>>();

@@ -151,6 +152,11 @@
      */

     private Properties clusterProps;

 

+    /**

+     * the Hyracks client connection

+     */

+    private IHyracksClientConnection hcc;

+

     public HyracksExecutionEngine(HiveConf conf) {

         this.conf = conf;

         init(conf);

@@ -533,17 +539,27 @@
             confIn.close();

         }

 

-        Process process = Runtime.getRuntime().exec("src/main/resources/scripts/getip.sh");

-        BufferedReader ipReader = new BufferedReader(new InputStreamReader(

-                new DataInputStream(process.getInputStream())));

-        String ipAddress = ipReader.readLine();

-        ipReader.close();

-        int port = Integer.parseInt(clusterProps.getProperty("CC_CLIENTPORT"));

-        String applicationName = "hivesterix";

+        if (hcc == null) {

+            BufferedReader ipReader = new BufferedReader(new InputStreamReader(new FileInputStream(masterFilePath)));

+            String masterNode = ipReader.readLine();

+            ipReader.close();

 

-        IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);

+            InetAddress[] ips = InetAddress.getAllByName(masterNode);

+            int port = Integer.parseInt(clusterProps.getProperty("CC_CLIENTPORT"));

+            for (InetAddress ip : ips) {

+                if (ip.getAddress().length <= 4) {

+                    try {

+                        hcc = new HyracksConnection(ip.getHostAddress(), port);

+                        break;

+                    } catch (Exception e) {

+                        continue;

+                    }

+                }

+            }

+        }

+

         long start = System.currentTimeMillis();

-        JobId jobId = hcc.startJob(applicationName, job);

+        JobId jobId = hcc.startJob(job);

         hcc.waitForCompletion(jobId);

 

         // System.out.println("job finished: " + jobId.toString());

diff --git a/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/LazyObjectInspectorFactory.java b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/LazyObjectInspectorFactory.java
new file mode 100644
index 0000000..7920001
--- /dev/null
+++ b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/LazyObjectInspectorFactory.java
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.lazy.objectinspector;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.io.Text;
+
+/**
+ * ObjectInspectorFactory is the primary way to create new ObjectInspector
+ * instances.
+ * SerDe classes should call the static functions in this library to create an
+ * ObjectInspector to return to the caller of SerDe2.getObjectInspector().
+ * The reason of having caches here is that ObjectInspectors do not have an
+ * internal state - so ObjectInspectors with the same construction parameters
+ * should result in exactly the same ObjectInspector.
+ */
+public final class LazyObjectInspectorFactory {
+
+    static ConcurrentHashMap<ArrayList<Object>, LazySimpleStructObjectInspector> cachedLazySimpleStructObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazySimpleStructObjectInspector>();
+
+    public static LazySimpleStructObjectInspector getLazySimpleStructObjectInspector(List<String> structFieldNames,
+            List<ObjectInspector> structFieldObjectInspectors, byte separator, Text nullSequence,
+            boolean lastColumnTakesRest, boolean escaped, byte escapeChar) {
+        ArrayList<Object> signature = new ArrayList<Object>();
+        signature.add(structFieldNames);
+        signature.add(structFieldObjectInspectors);
+        signature.add(Byte.valueOf(separator));
+        signature.add(nullSequence.toString());
+        signature.add(Boolean.valueOf(lastColumnTakesRest));
+        signature.add(Boolean.valueOf(escaped));
+        signature.add(Byte.valueOf(escapeChar));
+        LazySimpleStructObjectInspector result = cachedLazySimpleStructObjectInspector.get(signature);
+        if (result == null) {
+            result = new LazySimpleStructObjectInspector(structFieldNames, structFieldObjectInspectors, separator,
+                    nullSequence, lastColumnTakesRest, escaped, escapeChar);
+            cachedLazySimpleStructObjectInspector.put(signature, result);
+        }
+        return result;
+    }
+
+    static ConcurrentHashMap<ArrayList<Object>, LazyListObjectInspector> cachedLazySimpleListObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyListObjectInspector>();
+
+    public static LazyListObjectInspector getLazySimpleListObjectInspector(ObjectInspector listElementObjectInspector,
+            byte separator, Text nullSequence, boolean escaped, byte escapeChar) {
+        ArrayList<Object> signature = new ArrayList<Object>();
+        signature.add(listElementObjectInspector);
+        signature.add(Byte.valueOf(separator));
+        signature.add(nullSequence.toString());
+        signature.add(Boolean.valueOf(escaped));
+        signature.add(Byte.valueOf(escapeChar));
+        LazyListObjectInspector result = cachedLazySimpleListObjectInspector.get(signature);
+        if (result == null) {
+            result = new LazyListObjectInspector(listElementObjectInspector, separator, nullSequence, escaped,
+                    escapeChar);
+            cachedLazySimpleListObjectInspector.put(signature, result);
+        }
+        return result;
+    }
+
+    static ConcurrentHashMap<ArrayList<Object>, LazyMapObjectInspector> cachedLazySimpleMapObjectInspector = new ConcurrentHashMap<ArrayList<Object>, LazyMapObjectInspector>();
+
+    public static LazyMapObjectInspector getLazySimpleMapObjectInspector(ObjectInspector mapKeyObjectInspector,
+            ObjectInspector mapValueObjectInspector, byte itemSeparator, byte keyValueSeparator, Text nullSequence,
+            boolean escaped, byte escapeChar) {
+        ArrayList<Object> signature = new ArrayList<Object>();
+        signature.add(mapKeyObjectInspector);
+        signature.add(mapValueObjectInspector);
+        signature.add(Byte.valueOf(itemSeparator));
+        signature.add(Byte.valueOf(keyValueSeparator));
+        signature.add(nullSequence.toString());
+        signature.add(Boolean.valueOf(escaped));
+        signature.add(Byte.valueOf(escapeChar));
+        LazyMapObjectInspector result = cachedLazySimpleMapObjectInspector.get(signature);
+        if (result == null) {
+            result = new LazyMapObjectInspector(mapKeyObjectInspector, mapValueObjectInspector, itemSeparator,
+                    keyValueSeparator, nullSequence, escaped, escapeChar);
+            cachedLazySimpleMapObjectInspector.put(signature, result);
+        }
+        return result;
+    }
+
+    static ConcurrentHashMap<List<Object>, LazyUnionObjectInspector> cachedLazyUnionObjectInspector = new ConcurrentHashMap<List<Object>, LazyUnionObjectInspector>();
+
+    public static LazyUnionObjectInspector getLazyUnionObjectInspector(List<ObjectInspector> ois, byte separator,
+            Text nullSequence, boolean escaped, byte escapeChar) {
+        List<Object> signature = new ArrayList<Object>();
+        signature.add(ois);
+        signature.add(Byte.valueOf(separator));
+        signature.add(nullSequence.toString());
+        signature.add(Boolean.valueOf(escaped));
+        signature.add(Byte.valueOf(escapeChar));
+        LazyUnionObjectInspector result = cachedLazyUnionObjectInspector.get(signature);
+        if (result == null) {
+            result = new LazyUnionObjectInspector(ois, separator, nullSequence, escaped, escapeChar);
+            cachedLazyUnionObjectInspector.put(signature, result);
+        }
+        return result;
+    }
+
+    private LazyObjectInspectorFactory() {
+        // prevent instantiation
+    }
+}
\ No newline at end of file
diff --git a/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
new file mode 100644
index 0000000..95b999e
--- /dev/null
+++ b/hivesterix/hivesterix-dist/src/main/java/org/apache/hadoop/hive/serde2/typeinfo/TypeInfoFactory.java
@@ -0,0 +1,128 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.serde2.typeinfo;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.hive.serde.Constants;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+
+/**
+ * TypeInfoFactory can be used to create the TypeInfo object for any types.
+ * TypeInfo objects are all read-only so we can reuse them easily.
+ * TypeInfoFactory has internal cache to make sure we don't create 2 TypeInfo
+ * objects that represents the same type.
+ */
+public final class TypeInfoFactory {
+
+    static ConcurrentHashMap<String, TypeInfo> cachedPrimitiveTypeInfo = new ConcurrentHashMap<String, TypeInfo>();
+
+    private TypeInfoFactory() {
+        // prevent instantiation
+    }
+
+    public static TypeInfo getPrimitiveTypeInfo(String typeName) {
+        if (null == PrimitiveObjectInspectorUtils.getTypeEntryFromTypeName(typeName)) {
+            throw new RuntimeException("Cannot getPrimitiveTypeInfo for " + typeName);
+        }
+        TypeInfo result = cachedPrimitiveTypeInfo.get(typeName);
+        if (result == null) {
+            result = new PrimitiveTypeInfo(typeName);
+            cachedPrimitiveTypeInfo.put(typeName, result);
+        }
+        return result;
+    }
+
+    public static final TypeInfo voidTypeInfo = getPrimitiveTypeInfo(Constants.VOID_TYPE_NAME);
+    public static final TypeInfo booleanTypeInfo = getPrimitiveTypeInfo(Constants.BOOLEAN_TYPE_NAME);
+    public static final TypeInfo intTypeInfo = getPrimitiveTypeInfo(Constants.INT_TYPE_NAME);
+    public static final TypeInfo longTypeInfo = getPrimitiveTypeInfo(Constants.BIGINT_TYPE_NAME);
+    public static final TypeInfo stringTypeInfo = getPrimitiveTypeInfo(Constants.STRING_TYPE_NAME);
+    public static final TypeInfo floatTypeInfo = getPrimitiveTypeInfo(Constants.FLOAT_TYPE_NAME);
+    public static final TypeInfo doubleTypeInfo = getPrimitiveTypeInfo(Constants.DOUBLE_TYPE_NAME);
+    public static final TypeInfo byteTypeInfo = getPrimitiveTypeInfo(Constants.TINYINT_TYPE_NAME);
+    public static final TypeInfo shortTypeInfo = getPrimitiveTypeInfo(Constants.SMALLINT_TYPE_NAME);
+
+    public static final TypeInfo unknownTypeInfo = getPrimitiveTypeInfo("unknown");
+
+    public static TypeInfo getPrimitiveTypeInfoFromPrimitiveWritable(Class<?> clazz) {
+        String typeName = PrimitiveObjectInspectorUtils.getTypeNameFromPrimitiveWritable(clazz);
+        if (typeName == null) {
+            throw new RuntimeException("Internal error: Cannot get typeName for " + clazz);
+        }
+        return getPrimitiveTypeInfo(typeName);
+    }
+
+    public static TypeInfo getPrimitiveTypeInfoFromJavaPrimitive(Class<?> clazz) {
+        return getPrimitiveTypeInfo(PrimitiveObjectInspectorUtils.getTypeNameFromPrimitiveJava(clazz));
+    }
+
+    static ConcurrentHashMap<ArrayList<List<?>>, TypeInfo> cachedStructTypeInfo = new ConcurrentHashMap<ArrayList<List<?>>, TypeInfo>();
+
+    public static TypeInfo getStructTypeInfo(List<String> names, List<TypeInfo> typeInfos) {
+        ArrayList<List<?>> signature = new ArrayList<List<?>>(2);
+        signature.add(names);
+        signature.add(typeInfos);
+        TypeInfo result = cachedStructTypeInfo.get(signature);
+        if (result == null) {
+            result = new StructTypeInfo(names, typeInfos);
+            cachedStructTypeInfo.put(signature, result);
+        }
+        return result;
+    }
+
+    static ConcurrentHashMap<List<?>, TypeInfo> cachedUnionTypeInfo = new ConcurrentHashMap<List<?>, TypeInfo>();
+
+    public static TypeInfo getUnionTypeInfo(List<TypeInfo> typeInfos) {
+        TypeInfo result = cachedUnionTypeInfo.get(typeInfos);
+        if (result == null) {
+            result = new UnionTypeInfo(typeInfos);
+            cachedUnionTypeInfo.put(typeInfos, result);
+        }
+        return result;
+    }
+
+    static ConcurrentHashMap<TypeInfo, TypeInfo> cachedListTypeInfo = new ConcurrentHashMap<TypeInfo, TypeInfo>();
+
+    public static TypeInfo getListTypeInfo(TypeInfo elementTypeInfo) {
+        TypeInfo result = cachedListTypeInfo.get(elementTypeInfo);
+        if (result == null) {
+            result = new ListTypeInfo(elementTypeInfo);
+            cachedListTypeInfo.put(elementTypeInfo, result);
+        }
+        return result;
+    }
+
+    static ConcurrentHashMap<ArrayList<TypeInfo>, TypeInfo> cachedMapTypeInfo = new ConcurrentHashMap<ArrayList<TypeInfo>, TypeInfo>();
+
+    public static TypeInfo getMapTypeInfo(TypeInfo keyTypeInfo, TypeInfo valueTypeInfo) {
+        ArrayList<TypeInfo> signature = new ArrayList<TypeInfo>(2);
+        signature.add(keyTypeInfo);
+        signature.add(valueTypeInfo);
+        TypeInfo result = cachedMapTypeInfo.get(signature);
+        if (result == null) {
+            result = new MapTypeInfo(keyTypeInfo, valueTypeInfo);
+            cachedMapTypeInfo.put(signature, result);
+        }
+        return result;
+    };
+
+}
\ No newline at end of file
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh
index e0cdf73..8c9ae76 100755
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh
@@ -8,6 +8,10 @@
         IPADDR=`/sbin/ifconfig eth0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
 	if [ "$IPADDR" = "" ]
         then
+                IPADDR=`/sbin/ifconfig em1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+        fi
+	if [ "$IPADDR" = "" ]
+        then
 		IPADDR=`/sbin/ifconfig lo | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
         fi 
 else
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/startcc.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/startcc.sh
index fe2551d..efb79ce 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/startcc.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/startcc.sh
@@ -22,4 +22,4 @@
 
 #Launch hyracks cc script
 chmod -R 755 $HYRACKS_HOME
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 3 &> $CCLOGS_DIR/cc.log &
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/startnc.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/startnc.sh
index 6e0f90e..23a4c36 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/startnc.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/startnc.sh
@@ -46,4 +46,4 @@
 cd $NCTMP_DIR
 
 #Launch hyracks nc
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -result-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/stopnc.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/stopnc.sh
index 03ce4e7..35c4794 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/stopnc.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/stopnc.sh
@@ -5,6 +5,10 @@
 PID=`ps -ef|grep ${USER}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
 
 if [ "$PID" == "" ]; then
+  PID=`ps -ef|grep ${USER}|grep java|grep 'hyracks'|awk '{print $2}'`
+fi
+
+if [ "$PID" == "" ]; then
   USERID=`id | sed 's/^uid=//;s/(.*$//'`
   PID=`ps -ef|grep ${USERID}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
 fi
diff --git a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestSuiteClass.java b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestSuiteClass.java
index d9492d9..393378f 100644
--- a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestSuiteClass.java
+++ b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestSuiteClass.java
@@ -22,8 +22,6 @@
 import org.apache.hadoop.mapred.MiniMRCluster;

 

 import edu.uci.ics.hivesterix.common.config.ConfUtil;

-import edu.uci.ics.hyracks.api.client.HyracksConnection;

-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;

 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;

 import edu.uci.ics.hyracks.control.common.controllers.CCConfig;

 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;

@@ -93,7 +91,6 @@
         String ipAddress = hconf.get("hive.hyracks.host");

         int clientPort = Integer.parseInt(hconf.get("hive.hyracks.port"));

         int clusterPort = clientPort;

-        String applicationName = hconf.get("hive.hyracks.app");

 

         // start hyracks cc

         CCConfig ccConfig = new CCConfig();

@@ -113,14 +110,12 @@
             ncConfig.clusterNetIPAddress = ipAddress;

             ncConfig.ccPort = clientPort;

             ncConfig.dataIPAddress = "127.0.0.1";

+            ncConfig.datasetIPAddress = "127.0.0.1";

             ncConfig.nodeId = "nc" + i;

             NodeControllerService nc = new NodeControllerService(ncConfig);

             nc.start();

             ncs.put(ncConfig.nodeId, nc);

         }

-

-        IHyracksClientConnection hcc = new HyracksConnection(ccConfig.clientNetIpAddress, clientPort);

-        hcc.createApplication(applicationName, null);

     }

 

     protected void makeDir(String path) throws IOException {

diff --git a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractTestSuiteClass.java b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractTestSuiteClass.java
index e455527..72c406f 100644
--- a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractTestSuiteClass.java
+++ b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractTestSuiteClass.java
@@ -1,7 +1,6 @@
 package edu.uci.ics.hivesterix.test.base;

 

 import java.io.BufferedReader;

-import java.io.DataInputStream;

 import java.io.File;

 import java.io.FileInputStream;

 import java.io.FileNotFoundException;

@@ -9,6 +8,7 @@
 import java.io.IOException;

 import java.io.InputStream;

 import java.io.InputStreamReader;

+import java.net.InetAddress;

 import java.util.ArrayList;

 import java.util.HashMap;

 import java.util.Iterator;

@@ -28,8 +28,6 @@
 import org.apache.hadoop.mapred.MiniMRCluster;

 

 import edu.uci.ics.hivesterix.common.config.ConfUtil;

-import edu.uci.ics.hyracks.api.client.HyracksConnection;

-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;

 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;

 import edu.uci.ics.hyracks.control.common.controllers.CCConfig;

 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;

@@ -45,8 +43,9 @@
     private static final String PATH_TO_DATA = "src/test/resources/runtimefunctionts/data/";

 

     private static final String clusterPropertiesPath = "conf/cluster.properties";

-    private Properties clusterProps;

+    private static final String masterFilePath = "conf/master";

 

+    private Properties clusterProps;

     private MiniDFSCluster dfsCluster;

     private MiniMRCluster mrCluster;

 

@@ -109,14 +108,18 @@
             clusterProps.load(confIn);

             confIn.close();

         }

-        Process process = Runtime.getRuntime().exec("src/main/resources/scripts/getip.sh");

-        BufferedReader ipReader = new BufferedReader(new InputStreamReader(

-                new DataInputStream(process.getInputStream())));

-        String ipAddress = ipReader.readLine();

+        BufferedReader ipReader = new BufferedReader(new InputStreamReader(new FileInputStream(masterFilePath)));

+        String masterNode = ipReader.readLine();

         ipReader.close();

+        InetAddress[] ips = InetAddress.getAllByName(masterNode);

+        String ipAddress = null;

+        for (InetAddress ip : ips) {

+            if (ip.getAddress().length <= 4) {

+                ipAddress = ip.getHostAddress();

+            }

+        }

         int clientPort = Integer.parseInt(clusterProps.getProperty("CC_CLIENTPORT"));

         int netPort = Integer.parseInt(clusterProps.getProperty("CC_CLUSTERPORT"));

-        String applicationName = "hivesterix";

 

         // start hyracks cc

         CCConfig ccConfig = new CCConfig();

@@ -137,14 +140,12 @@
             ncConfig.clusterNetIPAddress = ipAddress;

             ncConfig.ccPort = netPort;

             ncConfig.dataIPAddress = "127.0.0.1";

+            ncConfig.datasetIPAddress = "127.0.0.1";

             ncConfig.nodeId = "nc" + i;

             NodeControllerService nc = new NodeControllerService(ncConfig);

             nc.start();

             ncs.put(ncConfig.nodeId, nc);

         }

-

-        IHyracksClientConnection hcc = new HyracksConnection(ccConfig.clientNetIpAddress, clientPort);

-        hcc.createApplication(applicationName, null);

     }

 

     protected void makeDir(String path) throws IOException {

diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/topology.xml b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/topology.xml
index 4aac091..4710706 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/topology.xml
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/topology.xml
@@ -1,12 +1,7 @@
 <cluster-topology>
-    <network-switch name="switch1">
-        <network-switch name="switch2">
-            <terminal name="nc0"/>
-            <terminal name="nc3"/>
-        </network-switch>
-        <network-switch name="switch3">
-            <terminal name="nc1"/>
-            <terminal name="nc4"/>
+    <network-switch name="Global">
+        <network-switch name="local">
+            <terminal name="127.0.0.1"/>
         </network-switch>
     </network-switch>
 </cluster-topology>
\ No newline at end of file
diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
index 4dfb1da..fc046fb 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
@@ -95,6 +95,13 @@
     }

 

     @Override

+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,

+            int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc, boolean ordered,

+            JobSpecification spec) throws AlgebricksException {

+        return null;

+    }

+

+    @Override

     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(IDataSource<S> arg0,

             IOperatorSchema arg1, List<LogicalVariable> arg2, LogicalVariable arg3, JobGenContext arg4,

             JobSpecification arg5) throws AlgebricksException {

diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveScanRuntimeGenerator.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveScanRuntimeGenerator.java
index 0d2c78a..5e4e21e 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveScanRuntimeGenerator.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveScanRuntimeGenerator.java
@@ -25,6 +25,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;

 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;

 import edu.uci.ics.hyracks.api.job.JobSpecification;

+import edu.uci.ics.hyracks.api.topology.ClusterTopology;

 import edu.uci.ics.hyracks.hdfs.dataflow.HDFSReadOperatorDescriptor;

 import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;

 

@@ -86,7 +87,8 @@
             JobConf conf = ConfUtil.getJobConf(fileDesc.getInputFileFormatClass(), filePath);

             String[] locConstraints = ConfUtil.getNCs();

             Map<String, NodeControllerInfo> ncNameToNcInfos = ConfUtil.getNodeControllerInfo();

-            Scheduler scheduler = new Scheduler(ncNameToNcInfos);

+            ClusterTopology topology = ConfUtil.getClusterTopology();

+            Scheduler scheduler = new Scheduler(ncNameToNcInfos, topology);

             InputSplit[] splits = conf.getInputFormat().getSplits(conf, locConstraints.length);

             String[] schedule = scheduler.getLocationConstraints(splits);

             IOperatorDescriptor scanner = new HDFSReadOperatorDescriptor(jobSpec, recDescriptor, conf, splits,

diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java
index 3a62b2e..81faf38 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/operator/filewrite/HiveFileWritePushRuntime.java
@@ -24,124 +24,130 @@
 @SuppressWarnings("deprecation")

 public class HiveFileWritePushRuntime implements IPushRuntime {

 

-    /**

-     * frame tuple accessor to access byte buffer

-     */

-    private final FrameTupleAccessor accessor;

+	/**

+	 * frame tuple accessor to access byte buffer

+	 */

+	private final FrameTupleAccessor accessor;

 

-    /**

-     * input object inspector

-     */

-    private final ObjectInspector inputInspector;

+	/**

+	 * input object inspector

+	 */

+	private final ObjectInspector inputInspector;

 

-    /**

-     * cachedInput

-     */

-    private final LazyColumnar cachedInput;

+	/**

+	 * cachedInput

+	 */

+	private final LazyColumnar cachedInput;

 

-    /**

-     * File sink operator of Hive

-     */

-    private final FileSinkDesc fileSink;

+	/**

+	 * File sink operator of Hive

+	 */

+	private final FileSinkDesc fileSink;

 

-    /**

-     * job configuration, which contain name node and other configuration

-     * information

-     */

-    private JobConf conf;

+	/**

+	 * job configuration, which contain name node and other configuration

+	 * information

+	 */

+	private JobConf conf;

 

-    /**

-     * input object inspector

-     */

-    private final Schema inputSchema;

+	/**

+	 * input object inspector

+	 */

+	private final Schema inputSchema;

 

-    /**

-     * a copy of hive schema representation

-     */

-    private RowSchema rowSchema;

+	/**

+	 * a copy of hive schema representation

+	 */

+	private RowSchema rowSchema;

 

-    /**

-     * the Hive file sink operator

-     */

-    private FileSinkOperator fsOp;

+	/**

+	 * the Hive file sink operator

+	 */

+	private FileSinkOperator fsOp;

 

-    /**

-     * cached tuple object reference

-     */

-    private FrameTupleReference tuple = new FrameTupleReference();

+	/**

+	 * cached tuple object reference

+	 */

+	private FrameTupleReference tuple = new FrameTupleReference();

 

-    /**

-     * @param spec

-     * @param fsProvider

-     */

-    public HiveFileWritePushRuntime(IHyracksTaskContext context, RecordDescriptor inputRecordDesc, JobConf job,

-            FileSinkDesc fs, RowSchema schema, Schema oi) {

-        fileSink = fs;

-        fileSink.setGatherStats(false);

+	/**

+	 * @param spec

+	 * @param fsProvider

+	 */

+	public HiveFileWritePushRuntime(IHyracksTaskContext context,

+			RecordDescriptor inputRecordDesc, JobConf job, FileSinkDesc fs,

+			RowSchema schema, Schema oi) {

+		fileSink = fs;

+		fileSink.setGatherStats(false);

 

-        rowSchema = schema;

-        conf = job;

-        inputSchema = oi;

+		rowSchema = schema;

+		conf = job;

+		inputSchema = oi;

 

-        accessor = new FrameTupleAccessor(context.getFrameSize(), inputRecordDesc);

-        inputInspector = inputSchema.toObjectInspector();

-        cachedInput = new LazyColumnar((LazyColumnarObjectInspector) inputInspector);

-    }

+		accessor = new FrameTupleAccessor(context.getFrameSize(),

+				inputRecordDesc);

+		inputInspector = inputSchema.toObjectInspector();

+		cachedInput = new LazyColumnar(

+				(LazyColumnarObjectInspector) inputInspector);

+	}

 

-    @Override

-    public void open() throws HyracksDataException {

-        fsOp = (FileSinkOperator) OperatorFactory.get(fileSink, rowSchema);

-        fsOp.setChildOperators(null);

-        fsOp.setParentOperators(null);

-        conf.setClassLoader(this.getClass().getClassLoader());

+	@Override

+	public void open() throws HyracksDataException {

+		fsOp = (FileSinkOperator) OperatorFactory.get(fileSink, rowSchema);

+		fsOp.setChildOperators(null);

+		fsOp.setParentOperators(null);

+		conf.setClassLoader(this.getClass().getClassLoader());

 

-        ObjectInspector[] inspectors = new ObjectInspector[1];

-        inspectors[0] = inputInspector;

-        try {

-            fsOp.initialize(conf, inspectors);

-            fsOp.setExecContext(null);

-        } catch (Exception e) {

-            e.printStackTrace();

-        }

-    }

+		ObjectInspector[] inspectors = new ObjectInspector[1];

+		inspectors[0] = inputInspector;

+		try {

+			fsOp.initialize(conf, inspectors);

+			fsOp.setExecContext(null);

+		} catch (Exception e) {

+			e.printStackTrace();

+		}

+	}

 

-    @Override

-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {

-        accessor.reset(buffer);

-        int n = accessor.getTupleCount();

-        try {

-            for (int i = 0; i < n; ++i) {

-                tuple.reset(accessor, i);

-                cachedInput.init(tuple);

-                fsOp.process(cachedInput, 0);

-            }

-        } catch (HiveException e) {

-            throw new HyracksDataException(e);

-        }

-    }

+	@Override

+	public void nextFrame(ByteBuffer buffer) throws HyracksDataException {

+		accessor.reset(buffer);

+		int n = accessor.getTupleCount();

+		try {

+			for (int i = 0; i < n; ++i) {

+				tuple.reset(accessor, i);

+				cachedInput.init(tuple);

+				fsOp.process(cachedInput, 0);

+			}

+		} catch (HiveException e) {

+			throw new HyracksDataException(e);

+		}

+	}

 

-    @Override

-    public void close() throws HyracksDataException {

-        try {

-            Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());

-            fsOp.closeOp(false);

-        } catch (HiveException e) {

-            throw new HyracksDataException(e);

-        }

-    }

+	@Override

+	public void close() throws HyracksDataException {

+		try {

+			Thread.currentThread().setContextClassLoader(

+					this.getClass().getClassLoader());

+			fsOp.closeOp(false);

+		} catch (HiveException e) {

+			throw new HyracksDataException(e);

+		}

+	}

 

-    @Override

-    public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {

-        throw new IllegalStateException();

-    }

+	@Override

+	public void setFrameWriter(int index, IFrameWriter writer,

+			RecordDescriptor recordDesc) {

+		throw new IllegalStateException();

+	}

 

-    @Override

-    public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {

-    }

+	@Override

+	public void setInputRecordDescriptor(int index,

+			RecordDescriptor recordDescriptor) {

+	}

 

-    @Override

-    public void fail() throws HyracksDataException {

+	@Override

+	public void fail() throws HyracksDataException {

 

-    }

+	}

 

 }

diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java
index 81cf511..1b079d6 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java
@@ -19,22 +19,12 @@
 import edu.uci.ics.hyracks.api.messages.IMessageBroker;
 
 /**
- * Base class of the {@link ICCApplicationContext} and the
- * {@link INCApplicationContext}.
+ * Base class of the {@link ICCApplicationContext} and the {@link INCApplicationContext}.
  * 
  * @author vinayakb
- * 
  */
 public interface IApplicationContext {
     /**
-     * Provides the Class Loader that loads classes for this Hyracks Application
-     * at the CC.
-     * 
-     * @return the application {@link ClassLoader}.
-     */
-    public ClassLoader getClassLoader();
-
-    /**
      * Gets the distributed state that is made available to all the Application
      * Contexts of this application in the cluster.
      * 
@@ -45,6 +35,4 @@
     public void setMessageBroker(IMessageBroker messageBroker);
 
     public IMessageBroker getMessageBroker();
-
-    public String getApplicationName();
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IBootstrap.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IBootstrap.java
deleted file mode 100644
index 19f97b4..0000000
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IBootstrap.java
+++ /dev/null
@@ -1,36 +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.application;
-
-/**
- * Base class of {@link ICCBootstrap} and {@link INCBootstrap}.
- * 
- * @author vinayakb
- */
-public interface IBootstrap {
-    /**
-     * Method called to start the application at a Hyracks CC or NC node.
-     * 
-     * @throws Exception
-     */
-    public void start() throws Exception;
-
-    /**
-     * Method called to shutdown the application at a Hyracks CC or NC node.
-     * 
-     * @throws Exception
-     */
-    public void stop() throws Exception;
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
index 2792d29..c4b7802 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
@@ -27,8 +27,10 @@
 public interface ICCApplicationContext extends IApplicationContext {
     /**
      * Sets the state that must be distributed by the infrastructure to all the
-     * NC application contects. Any state set by calling thsi method in the {@link ICCBootstrap#start()} call is made available to all the {@link INCApplicationContext} objects at each Node Controller. The state
-     * is then available to be inspected by the application at the NC during or
+     * NC application contexts. Any state set by calling this method in
+     * the {@link ICCApplicationEntryPoint#start(ICCApplicationContext, String[])} call
+     * is made available to all the {@link INCApplicationContext} objects at each Node Controller.
+     * The state is then available to be inspected by the application at the NC during or
      * after the {@link INCBootstrap#start()} call.
      * 
      * @param state
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationEntryPoint.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationEntryPoint.java
new file mode 100644
index 0000000..c22cc51
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationEntryPoint.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.hyracks.api.application;
+
+public interface ICCApplicationEntryPoint {
+    public void start(ICCApplicationContext ccAppCtx, String[] args) throws Exception;
+
+    public void stop() throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCBootstrap.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCBootstrap.java
deleted file mode 100644
index e3906ea..0000000
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCBootstrap.java
+++ /dev/null
@@ -1,19 +0,0 @@
-package edu.uci.ics.hyracks.api.application;
-
-/**
- * Implemented by the bootstrap class of the application that will manage its
- * life cycle at the Cluster Controller.
- * 
- * @author vinayakb
- * 
- */
-public interface ICCBootstrap extends IBootstrap {
-    /**
-     * Called by the infrastructure to set the CC Application Context for the
-     * application. The infrastructure makes this call prior to calling start().
-     * 
-     * @param appCtx
-     *            - The CC application context
-     */
-    public void setApplicationContext(ICCApplicationContext appCtx);
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationEntryPoint.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationEntryPoint.java
new file mode 100644
index 0000000..9a25b3c
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationEntryPoint.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.hyracks.api.application;
+
+public interface INCApplicationEntryPoint {
+    public void start(INCApplicationContext ncAppCtx, String[] args) throws Exception;
+
+    public void notifyStartupComplete() throws Exception;
+
+    public void stop() throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCBootstrap.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCBootstrap.java
deleted file mode 100644
index 300f7c7..0000000
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCBootstrap.java
+++ /dev/null
@@ -1,19 +0,0 @@
-package edu.uci.ics.hyracks.api.application;
-
-/**
- * Implemented by the bootstrap class of the application that will manage its
- * life cycle at a Node Controller.
- * 
- * @author vinayakb
- * 
- */
-public interface INCBootstrap extends IBootstrap {
-    /**
-     * Called by the infrastructure to set the NC Application Context for the
-     * application. The infrastructure makes this call prior to calling start().
-     * 
-     * @param appCtx
-     *            - The NC application context
-     */
-    public void setApplicationContext(INCApplicationContext appCtx);
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
index a8f2fda..a4f0b29 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface IInputChannel {
@@ -30,7 +30,7 @@
 
     public void recycleBuffer(ByteBuffer buffer);
 
-    public void open(IHyracksTaskContext ctx) throws HyracksDataException;
+    public void open(IHyracksCommonContext ctx) throws HyracksDataException;
 
     public void close() throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
index e34e60d..88df49f 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -17,6 +17,8 @@
 import java.io.Serializable;
 import java.util.EnumSet;
 
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 
@@ -24,12 +26,13 @@
     public enum FunctionId {
         GET_CLUSTER_CONTROLLER_INFO,
         GET_CLUSTER_TOPOLOGY,
-        CREATE_APPLICATION,
-        START_APPLICATION,
-        DESTROY_APPLICATION,
         CREATE_JOB,
         GET_JOB_STATUS,
         START_JOB,
+        GET_DATASET_DIRECTORY_SERIVICE_INFO,
+        GET_DATASET_RESULT_STATUS,
+        GET_DATASET_RECORD_DESCRIPTOR,
+        GET_DATASET_RESULT_LOCATIONS,
         WAIT_FOR_COMPLETION,
         GET_NODE_CONTROLLERS_INFO
     }
@@ -49,63 +52,6 @@
         }
     }
 
-    public static class CreateApplicationFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final String appName;
-
-        public CreateApplicationFunction(String appName) {
-            this.appName = appName;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.CREATE_APPLICATION;
-        }
-
-        public String getAppName() {
-            return appName;
-        }
-    }
-
-    public static class StartApplicationFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final String appName;
-
-        public StartApplicationFunction(String appName) {
-            this.appName = appName;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.START_APPLICATION;
-        }
-
-        public String getAppName() {
-            return appName;
-        }
-    }
-
-    public static class DestroyApplicationFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final String appName;
-
-        public DestroyApplicationFunction(String appName) {
-            this.appName = appName;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.DESTROY_APPLICATION;
-        }
-
-        public String getAppName() {
-            return appName;
-        }
-    }
-
     public static class GetJobStatusFunction extends Function {
         private static final long serialVersionUID = 1L;
 
@@ -128,12 +74,10 @@
     public static class StartJobFunction extends Function {
         private static final long serialVersionUID = 1L;
 
-        private final String appName;
         private final byte[] acggfBytes;
         private final EnumSet<JobFlag> jobFlags;
 
-        public StartJobFunction(String appName, byte[] acggfBytes, EnumSet<JobFlag> jobFlags) {
-            this.appName = appName;
+        public StartJobFunction(byte[] acggfBytes, EnumSet<JobFlag> jobFlags) {
             this.acggfBytes = acggfBytes;
             this.jobFlags = jobFlags;
         }
@@ -143,10 +87,6 @@
             return FunctionId.START_JOB;
         }
 
-        public String getAppName() {
-            return appName;
-        }
-
         public byte[] getACGGFBytes() {
             return acggfBytes;
         }
@@ -156,6 +96,74 @@
         }
     }
 
+    public static class GetDatasetDirectoryServiceInfoFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.GET_DATASET_DIRECTORY_SERIVICE_INFO;
+        }
+    }
+
+    public static class GetDatasetResultStatusFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+
+        private final ResultSetId rsId;
+
+        public GetDatasetResultStatusFunction(JobId jobId, ResultSetId rsId) {
+            this.jobId = jobId;
+            this.rsId = rsId;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.GET_DATASET_RESULT_STATUS;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public ResultSetId getResultSetId() {
+            return rsId;
+        }
+    }
+
+    public static class GetDatasetResultLocationsFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+
+        private final ResultSetId rsId;
+
+        private final DatasetDirectoryRecord[] knownRecords;
+
+        public GetDatasetResultLocationsFunction(JobId jobId, ResultSetId rsId, DatasetDirectoryRecord[] knownRecords) {
+            this.jobId = jobId;
+            this.rsId = rsId;
+            this.knownRecords = knownRecords;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.GET_DATASET_RESULT_LOCATIONS;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public ResultSetId getResultSetId() {
+            return rsId;
+        }
+
+        public DatasetDirectoryRecord[] getKnownRecords() {
+            return knownRecords;
+        }
+    }
+
     public static class WaitForCompletionFunction extends Function {
         private static final long serialVersionUID = 1L;
 
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
index 4c06d42..033fc02 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
@@ -17,6 +17,7 @@
 import java.util.EnumSet;
 import java.util.Map;
 
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -41,27 +42,6 @@
     }
 
     @Override
-    public void createApplication(String appName) throws Exception {
-        HyracksClientInterfaceFunctions.CreateApplicationFunction caf = new HyracksClientInterfaceFunctions.CreateApplicationFunction(
-                appName);
-        rpci.call(ipcHandle, caf);
-    }
-
-    @Override
-    public void startApplication(String appName) throws Exception {
-        HyracksClientInterfaceFunctions.StartApplicationFunction saf = new HyracksClientInterfaceFunctions.StartApplicationFunction(
-                appName);
-        rpci.call(ipcHandle, saf);
-    }
-
-    @Override
-    public void destroyApplication(String appName) throws Exception {
-        HyracksClientInterfaceFunctions.DestroyApplicationFunction daf = new HyracksClientInterfaceFunctions.DestroyApplicationFunction(
-                appName);
-        rpci.call(ipcHandle, daf);
-    }
-
-    @Override
     public JobStatus getJobStatus(JobId jobId) throws Exception {
         HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf = new HyracksClientInterfaceFunctions.GetJobStatusFunction(
                 jobId);
@@ -69,13 +49,19 @@
     }
 
     @Override
-    public JobId startJob(String appName, byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception {
+    public JobId startJob(byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception {
         HyracksClientInterfaceFunctions.StartJobFunction sjf = new HyracksClientInterfaceFunctions.StartJobFunction(
-                appName, acggfBytes, jobFlags);
+                acggfBytes, jobFlags);
         return (JobId) rpci.call(ipcHandle, sjf);
     }
 
     @Override
+    public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception {
+        HyracksClientInterfaceFunctions.GetDatasetDirectoryServiceInfoFunction gddsf = new HyracksClientInterfaceFunctions.GetDatasetDirectoryServiceInfoFunction();
+        return (NetworkAddress) rpci.call(ipcHandle, gddsf);
+    }
+
+    @Override
     public void waitForCompletion(JobId jobId) throws Exception {
         HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = new HyracksClientInterfaceFunctions.WaitForCompletionFunction(
                 jobId);
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
index 227524c..8274416 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
@@ -14,19 +14,12 @@
  */
 package edu.uci.ics.hyracks.api.client;
 
-import java.io.File;
 import java.net.InetSocketAddress;
 import java.util.EnumSet;
 import java.util.Map;
 
-import org.apache.http.HttpResponse;
-import org.apache.http.client.HttpClient;
-import org.apache.http.client.methods.HttpPut;
-import org.apache.http.entity.FileEntity;
-import org.apache.http.impl.client.DefaultHttpClient;
-
 import edu.uci.ics.hyracks.api.client.impl.JobSpecificationActivityClusterGraphGeneratorFactory;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
@@ -76,46 +69,28 @@
     }
 
     @Override
-    public void createApplication(String appName, File harFile) throws Exception {
-        hci.createApplication(appName);
-        if (harFile != null) {
-            HttpClient hc = new DefaultHttpClient();
-            HttpPut put = new HttpPut("http://" + ccHost + ":" + ccInfo.getWebPort() + "/applications/" + appName);
-            put.setEntity(new FileEntity(harFile, "application/octet-stream"));
-            HttpResponse response = hc.execute(put);
-            if (response.getStatusLine().getStatusCode() != 200) {
-                hci.destroyApplication(appName);
-                throw new HyracksException(response.getStatusLine().toString());
-            }
-        }
-        hci.startApplication(appName);
-    }
-
-    @Override
-    public void destroyApplication(String appName) throws Exception {
-        hci.destroyApplication(appName);
-    }
-
-    @Override
     public JobStatus getJobStatus(JobId jobId) throws Exception {
         return hci.getJobStatus(jobId);
     }
 
     @Override
-    public JobId startJob(String appName, JobSpecification jobSpec) throws Exception {
-        return startJob(appName, jobSpec, EnumSet.noneOf(JobFlag.class));
+    public JobId startJob(JobSpecification jobSpec) throws Exception {
+        return startJob(jobSpec, EnumSet.noneOf(JobFlag.class));
     }
 
     @Override
-    public JobId startJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+    public JobId startJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
         JobSpecificationActivityClusterGraphGeneratorFactory jsacggf = new JobSpecificationActivityClusterGraphGeneratorFactory(
                 jobSpec);
-        return startJob(appName, jsacggf, jobFlags);
+        return startJob(jsacggf, jobFlags);
     }
 
-    public JobId startJob(String appName, IActivityClusterGraphGeneratorFactory acggf, EnumSet<JobFlag> jobFlags)
-            throws Exception {
-        return hci.startJob(appName, JavaSerializationUtils.serialize(acggf), jobFlags);
+    public JobId startJob(IActivityClusterGraphGeneratorFactory acggf, EnumSet<JobFlag> jobFlags) throws Exception {
+        return hci.startJob(JavaSerializationUtils.serialize(acggf), jobFlags);
+    }
+
+    public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception {
+        return hci.getDatasetDirectoryServiceInfo();
     }
 
     @Override
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
index bdbb544..fe4094f 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
@@ -14,10 +14,10 @@
  */
 package edu.uci.ics.hyracks.api.client;
 
-import java.io.File;
 import java.util.EnumSet;
 import java.util.Map;
 
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
@@ -32,26 +32,6 @@
  */
 public interface IHyracksClientConnection {
     /**
-     * Create a Hyracks Application
-     * 
-     * @param appName
-     *            Name of the application
-     * @param harFile
-     *            Archive that contains deployable code for the application
-     * @throws Exception
-     */
-    public void createApplication(String appName, File harFile) throws Exception;
-
-    /**
-     * Destroy an already-deployed Hyracks application
-     * 
-     * @param appName
-     *            Name of the application
-     * @throws Exception
-     */
-    public void destroyApplication(String appName) throws Exception;
-
-    /**
      * Gets the status of the specified Job.
      * 
      * @param jobId
@@ -70,7 +50,7 @@
      *            Job Specification
      * @throws Exception
      */
-    public JobId startJob(String appName, JobSpecification jobSpec) throws Exception;
+    public JobId startJob(JobSpecification jobSpec) throws Exception;
 
     /**
      * Start the specified Job.
@@ -83,7 +63,7 @@
      *            Flags
      * @throws Exception
      */
-    public JobId startJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
+    public JobId startJob(JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
 
     /**
      * Start the specified Job.
@@ -96,8 +76,15 @@
      *            Flags
      * @throws Exception
      */
-    public JobId startJob(String appName, IActivityClusterGraphGeneratorFactory acggf, EnumSet<JobFlag> jobFlags)
-            throws Exception;
+    public JobId startJob(IActivityClusterGraphGeneratorFactory acggf, EnumSet<JobFlag> jobFlags) throws Exception;
+
+    /**
+     * Gets the IP Address and port for the DatasetDirectoryService wrapped in NetworkAddress
+     * 
+     * @return {@link NetworkAddress}
+     * @throws Exception
+     */
+    public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
 
     /**
      * Waits until the specified job has completed, either successfully or has
@@ -123,4 +110,4 @@
      * @throws Exception
      */
     public ClusterTopology getClusterTopology() throws Exception;
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
index ef5906e..6fdf638 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
@@ -17,6 +17,7 @@
 import java.util.EnumSet;
 import java.util.Map;
 
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -25,15 +26,11 @@
 public interface IHyracksClientInterface {
     public ClusterControllerInfo getClusterControllerInfo() throws Exception;
 
-    public void createApplication(String appName) throws Exception;
-
-    public void startApplication(String appName) throws Exception;
-
-    public void destroyApplication(String appName) throws Exception;
-
     public JobStatus getJobStatus(JobId jobId) throws Exception;
 
-    public JobId startJob(String appName, byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception;
+    public JobId startJob(byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception;
+
+    public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
 
     public void waitForCompletion(JobId jobId) throws Exception;
 
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
index fd9218a..73b5488 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
@@ -27,10 +27,14 @@
 
     private final NetworkAddress netAddress;
 
-    public NodeControllerInfo(String nodeId, NodeStatus status, NetworkAddress netAddress) {
+    private final NetworkAddress datasetNetworkAddress;
+
+    public NodeControllerInfo(String nodeId, NodeStatus status, NetworkAddress netAddress,
+            NetworkAddress datasetNetworkAddress) {
         this.nodeId = nodeId;
         this.status = status;
         this.netAddress = netAddress;
+        this.datasetNetworkAddress = datasetNetworkAddress;
     }
 
     public String getNodeId() {
@@ -44,4 +48,8 @@
     public NetworkAddress getNetworkAddress() {
         return netAddress;
     }
+
+    public NetworkAddress getDatasetNetworkAddress() {
+        return datasetNetworkAddress;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
index 0eac9a2..3431c40 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
@@ -29,7 +29,7 @@
     }
 
     @Override
-    public IActivityClusterGraphGenerator createActivityClusterGraphGenerator(String appName, JobId jobId,
+    public IActivityClusterGraphGenerator createActivityClusterGraphGenerator(JobId jobId,
             final ICCApplicationContext ccAppCtx, EnumSet<JobFlag> jobFlags) throws HyracksException {
         final JobActivityGraphBuilder builder = new JobActivityGraphBuilder(spec, jobFlags);
         PlanUtils.visit(spec, new IConnectorDescriptorVisitor() {
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
index e964d66..a2ee977 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
@@ -15,6 +15,7 @@
 package edu.uci.ics.hyracks.api.context;
 
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
 import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
@@ -28,5 +29,7 @@
 
     public ICounterContext getCounterContext();
 
+    public IDatasetPartitionManager getDatasetPartitionManager();
+
     public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializer.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializer.java
new file mode 100644
index 0000000..ba2ff9a
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializer.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataflow.value;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IResultSerializer extends Serializable {
+    /**
+     * Initializes the serializer.
+     */
+    public void init() throws HyracksDataException;
+
+    /**
+     * Method to serialize the result and append it to the provided output stream
+     * 
+     * @param tAccess
+     *            - A frame tuple accessor object that contains the original data to be serialized
+     * @param tIdx
+     *            - Index of the tuple that should be serialized.
+     * @return true if the tuple was appended successfully, else false.
+     */
+    public boolean appendTuple(IFrameTupleAccessor tAccess, int tIdx) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializerFactory.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializerFactory.java
new file mode 100644
index 0000000..1fbf00f
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializerFactory.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataflow.value;
+
+import java.io.PrintStream;
+import java.io.Serializable;
+
+public interface IResultSerializerFactory extends Serializable {
+    /**
+     * Creates a result serialized appender
+     * 
+     * @param printStream
+     *            - A print stream object to which the serialized results will be written.
+     * @return A new instance of result serialized appender.
+     */
+    public IResultSerializer createResultSerializer(RecordDescriptor recordDesc, PrintStream printStream);
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/JSONSerializable.java
similarity index 64%
copy from hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/JSONSerializable.java
index 5f7ef0b..1eca502 100644
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/JSONSerializable.java
@@ -12,11 +12,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.cli;
+package edu.uci.ics.hyracks.api.dataflow.value;
 
-public class Main {
-    public static void main(String[] args) throws Exception {
-        CLI cli = new CLI(args);
-        cli.run();
-    }
+import org.json.JSONException;
+import org.json.JSONObject;
+
+public interface JSONSerializable {
+    /**
+     * Returns the JSON representation of the object.
+     * 
+     * @return A new JSONObject instance representing this Java object.
+     */
+    public JSONObject toJSON() throws JSONException;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/DatasetDirectoryRecord.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/DatasetDirectoryRecord.java
new file mode 100644
index 0000000..6316bba
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/DatasetDirectoryRecord.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+
+public class DatasetDirectoryRecord implements Serializable {
+    public enum Status {
+        IDLE,
+        RUNNING,
+        SUCCESS,
+        FAILED
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    private NetworkAddress address;
+
+    private boolean readEOS;
+
+    private Status status;
+
+    public DatasetDirectoryRecord() {
+        this.address = null;
+        this.readEOS = false;
+        this.status = Status.IDLE;
+    }
+
+    public void setNetworkAddress(NetworkAddress address) {
+        this.address = address;
+    }
+
+    public NetworkAddress getNetworkAddress() {
+        return address;
+    }
+
+    public void readEOS() {
+        this.readEOS = true;
+    }
+
+    public boolean hasReachedReadEOS() {
+        return readEOS;
+    }
+
+    public void start() {
+        status = Status.RUNNING;
+    }
+
+    public void writeEOS() {
+        status = Status.SUCCESS;
+    }
+
+    public void fail() {
+        status = Status.FAILED;
+    }
+
+    public Status getStatus() {
+        return status;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof DatasetDirectoryRecord)) {
+            return false;
+        }
+        return address.equals(((DatasetDirectoryRecord) o).address);
+    }
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetDirectoryService.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetDirectoryService.java
new file mode 100644
index 0000000..5266333
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetDirectoryService.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IDatasetDirectoryService {
+    public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
+            int nPartitions, NetworkAddress networkAddress);
+
+    public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition);
+
+    public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition);
+
+    public Status getResultStatus(JobId jobId, ResultSetId rsId) throws HyracksDataException;
+
+    public DatasetDirectoryRecord[] getResultPartitionLocations(JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownLocations) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetInputChannelMonitor.java
similarity index 66%
copy from hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetInputChannelMonitor.java
index 5f7ef0b..65ba1c7 100644
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetInputChannelMonitor.java
@@ -12,11 +12,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.cli;
+package edu.uci.ics.hyracks.api.dataset;
 
-public class Main {
-    public static void main(String[] args) throws Exception {
-        CLI cli = new CLI(args);
-        cli.run();
-    }
-}
\ No newline at end of file
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+
+public interface IDatasetInputChannelMonitor extends IInputChannelMonitor {
+    public boolean eosReached();
+
+    public boolean failed();
+
+    public int getNFramesAvailable();
+
+    public void notifyFrameRead();
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java
new file mode 100644
index 0000000..ae38c7f
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IDatasetPartitionManager {
+    public IFrameWriter createDatasetPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
+            int partition, int nPartitions) throws HyracksException;
+
+    public void reportPartitionWriteCompletion(JobId jobId, ResultSetId resultSetId, int partition)
+            throws HyracksException;
+
+    public void reportPartitionFailure(JobId jobId, ResultSetId resultSetId, int partition) throws HyracksException;
+
+    public void initializeDatasetPartitionReader(JobId jobId, int partition, IFrameWriter noc) throws HyracksException;
+
+    public IWorkspaceFileFactory getFileFactory();
+
+    public void close();
+}
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java
similarity index 76%
rename from hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
rename to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java
index 5f7ef0b..8f5ed64 100644
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java
@@ -3,20 +3,19 @@
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * you may obtain a copy of the License from
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.cli;
+package edu.uci.ics.hyracks.api.dataset;
 
-public class Main {
-    public static void main(String[] args) throws Exception {
-        CLI cli = new CLI(args);
-        cli.run();
-    }
-}
\ No newline at end of file
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+
+public interface IDatasetPartitionReader {
+    public void writeTo(IFrameWriter writer);
+}
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionWriter.java
similarity index 68%
copy from hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionWriter.java
index 5f7ef0b..42dc157 100644
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionWriter.java
@@ -3,20 +3,20 @@
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * you may obtain a copy of the License from
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.cli;
+package edu.uci.ics.hyracks.api.dataset;
 
-public class Main {
-    public static void main(String[] args) throws Exception {
-        CLI cli = new CLI(args);
-        cli.run();
-    }
-}
\ No newline at end of file
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IDatasetPartitionWriter extends IFrameWriter {
+    public Page returnPage() throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDataset.java
similarity index 67%
copy from hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDataset.java
index 5f7ef0b..4a7a6b0 100644
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDataset.java
@@ -12,11 +12,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.cli;
+package edu.uci.ics.hyracks.api.dataset;
 
-public class Main {
-    public static void main(String[] args) throws Exception {
-        CLI cli = new CLI(args);
-        cli.run();
-    }
-}
\ No newline at end of file
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IHyracksDataset {
+    public IHyracksDatasetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java
new file mode 100644
index 0000000..d49d5cd
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IHyracksDatasetDirectoryServiceConnection {
+    /**
+     * Gets the result status for the given result set.
+     * 
+     * @param jobId
+     *            ID of the job
+     * @param rsId
+     *            ID of the result set
+     * @return {@link Status}
+     * @throws Exception
+     */
+    public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception;
+
+    /**
+     * Gets the IP Addresses and ports for the partition generating the result for each location.
+     * 
+     * @param jobId
+     *            ID of the job
+     * @param rsId
+     *            ID of the result set
+     * @param knownRecords
+     *            Locations that are already known to the client
+     * @return {@link NetworkAddress[]}
+     * @throws Exception
+     */
+    public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownRecords) throws Exception;
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java
new file mode 100644
index 0000000..ba21a84
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IHyracksDatasetDirectoryServiceInterface {
+    /**
+     * Gets the result status for the given result set.
+     * 
+     * @param jobId
+     *            ID of the job
+     * @param rsId
+     *            ID of the result set
+     * @return {@link Status}
+     * @throws Exception
+     */
+    public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception;
+
+    /**
+     * Gets the IP Addresses and ports for the partition generating the result for each location.
+     * 
+     * @param jobId
+     *            ID of the job
+     * @param rsId
+     *            ID of the result set
+     * @param knownRecords
+     *            Locations from the dataset directory that are already known to the client
+     * @return {@link NetworkAddress[]}
+     * @throws Exception
+     */
+    public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownRecords) throws Exception;
+}
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
similarity index 63%
copy from hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
index 5f7ef0b..b928a49 100644
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
@@ -12,11 +12,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.cli;
+package edu.uci.ics.hyracks.api.dataset;
 
-public class Main {
-    public static void main(String[] args) throws Exception {
-        CLI cli = new CLI(args);
-        cli.run();
-    }
-}
\ No newline at end of file
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IHyracksDatasetReader {
+    public Status getResultStatus();
+
+    public int read(ByteBuffer buffer) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/Page.java
similarity index 64%
copy from hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/Page.java
index 5f7ef0b..7275dfd 100644
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/Page.java
@@ -3,20 +3,31 @@
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * you may obtain a copy of the License from
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.cli;
+package edu.uci.ics.hyracks.api.dataset;
 
-public class Main {
-    public static void main(String[] args) throws Exception {
-        CLI cli = new CLI(args);
-        cli.run();
+import java.nio.ByteBuffer;
+
+public class Page {
+    private final ByteBuffer buffer;
+
+    public Page(ByteBuffer buffer) {
+        this.buffer = buffer;
     }
-}
\ No newline at end of file
+
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    public ByteBuffer clear() {
+        return (ByteBuffer) buffer.clear();
+    }
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/ResultSetId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/ResultSetId.java
new file mode 100644
index 0000000..ae38ef3
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/ResultSetId.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import java.io.Serializable;
+
+public class ResultSetId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final long id;
+
+    public ResultSetId(long id) {
+        this.id = id;
+    }
+
+    public long getId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int) id;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof ResultSetId)) {
+            return false;
+        }
+        return ((ResultSetId) o).id == id;
+    }
+
+    @Override
+    public String toString() {
+        return "RSID:" + id;
+    }
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IActivityClusterGraphGeneratorFactory.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IActivityClusterGraphGeneratorFactory.java
index d801dd1..ae76455 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IActivityClusterGraphGeneratorFactory.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IActivityClusterGraphGeneratorFactory.java
@@ -21,6 +21,6 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 
 public interface IActivityClusterGraphGeneratorFactory extends Serializable {
-    public IActivityClusterGraphGenerator createActivityClusterGraphGenerator(String appName, JobId jobId,
+    public IActivityClusterGraphGenerator createActivityClusterGraphGenerator(JobId jobId,
             ICCApplicationContext ccAppCtx, EnumSet<JobFlag> jobFlags) throws HyracksException;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
index 7c523f1..1fdff0f 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
@@ -34,12 +34,15 @@
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 
 public class JobSpecification implements Serializable, IOperatorDescriptorRegistry, IConnectorDescriptorRegistry {
     private static final long serialVersionUID = 1L;
 
     private final List<OperatorDescriptorId> roots;
 
+    private final List<ResultSetId> resultSetIds;
+
     private final Map<OperatorDescriptorId, IOperatorDescriptor> opMap;
 
     private final Map<ConnectorDescriptorId, IConnectorDescriptor> connMap;
@@ -72,6 +75,7 @@
 
     public JobSpecification() {
         roots = new ArrayList<OperatorDescriptorId>();
+        resultSetIds = new ArrayList<ResultSetId>();
         opMap = new HashMap<OperatorDescriptorId, IOperatorDescriptor>();
         connMap = new HashMap<ConnectorDescriptorId, IConnectorDescriptor>();
         opInputMap = new HashMap<OperatorDescriptorId, List<IConnectorDescriptor>>();
@@ -104,6 +108,10 @@
         roots.add(op.getOperatorId());
     }
 
+    public void addResultSetId(ResultSetId rsId) {
+        resultSetIds.add(rsId);
+    }
+
     public void connect(IConnectorDescriptor conn, IOperatorDescriptor producerOp, int producerPort,
             IOperatorDescriptor consumerOp, int consumerPort) {
         insertIntoIndexedMap(opInputMap, consumerOp.getOperatorId(), consumerPort, conn);
@@ -208,6 +216,10 @@
         return roots;
     }
 
+    public List<ResultSetId> getResultSetIds() {
+        return resultSetIds;
+    }
+
     public IConnectorPolicyAssignmentPolicy getConnectorPolicyAssignmentPolicy() {
         return connectorPolicyAssignmentPolicy;
     }
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/ResultSetPartitionId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/ResultSetPartitionId.java
new file mode 100644
index 0000000..148a8a2
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/ResultSetPartitionId.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.partitions;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public final class ResultSetPartitionId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final JobId jobId;
+
+    private final ResultSetId resultSetId;
+    
+    private final int partition;
+
+    public ResultSetPartitionId(JobId jobId, ResultSetId resultSetId, int partition) {
+        this.jobId = jobId;
+        this.resultSetId = resultSetId;
+        this.partition = partition;
+    }
+
+    public JobId getJobId() {
+        return jobId;
+    }
+
+    public ResultSetId getResultSetId() {
+        return resultSetId;
+    }
+
+    public int getPartition() {
+        return partition;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((resultSetId == null) ? 0 : resultSetId.hashCode());
+        result = prime * result + ((jobId == null) ? 0 : jobId.hashCode());
+        result = prime * result + partition;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        ResultSetPartitionId other = (ResultSetPartitionId) obj;
+        if (resultSetId == null) {
+            if (other.resultSetId != null)
+                return false;
+        } else if (!resultSetId.equals(other.resultSetId))
+            return false;
+        if (jobId == null) {
+            if (other.jobId != null)
+                return false;
+        } else if (!jobId.equals(other.jobId))
+            return false;
+        if (partition != other.partition)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return jobId.toString() + ":" + resultSetId + ":" + partition;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
index eba5560..7d50fa0 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
@@ -36,8 +36,8 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * The runtime of a SuperActivity, which internally executes a DAG of one-to-one connected
- * activities in a single thread.
+ * The runtime of a SuperActivity, which internally executes a DAG of one-to-one
+ * connected activities in a single thread.
  * 
  * @author yingyib
  */
@@ -62,7 +62,8 @@
         this.nPartitions = nPartitions;
 
         /**
-         * initialize the writer-relationship for the internal DAG of operator node pushables
+         * initialize the writer-relationship for the internal DAG of operator
+         * node pushables
          */
         try {
             init();
@@ -72,7 +73,7 @@
     }
 
     @Override
-    public synchronized void initialize() throws HyracksDataException {
+    public void initialize() throws HyracksDataException {
         /**
          * initialize operator node pushables in the BFS order
          */
@@ -105,7 +106,8 @@
         }
 
         /**
-         * Using BFS (breadth-first search) to construct to runtime execution DAG;
+         * Using BFS (breadth-first search) to construct to runtime execution
+         * DAG;
          */
         while (childQueue.size() > 0) {
             /**
@@ -165,8 +167,7 @@
     }
 
     @Override
-    public synchronized void setOutputFrameWriter(int clusterOutputIndex, IFrameWriter writer,
-            RecordDescriptor recordDesc) {
+    public void setOutputFrameWriter(int clusterOutputIndex, IFrameWriter writer, RecordDescriptor recordDesc) {
         /**
          * set the right output frame writer
          */
@@ -176,7 +177,7 @@
     }
 
     @Override
-    public synchronized IFrameWriter getInputFrameWriter(final int index) {
+    public IFrameWriter getInputFrameWriter(final int index) {
         /**
          * get the right IFrameWriter from the cluster input index
          */
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java
index 1185b5e..764348f 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java
@@ -33,6 +33,14 @@
         return baos.toByteArray();
     }
 
+    public static Object deserialize(byte[] bytes) throws IOException, ClassNotFoundException {
+        if (bytes == null) {
+            return null;
+        }
+        ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(bytes));
+        return ois.readObject();
+    }
+
     public static Object deserialize(byte[] bytes, ClassLoader classLoader) throws IOException, ClassNotFoundException {
         if (bytes == null) {
             return null;
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CLI.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CLI.java
deleted file mode 100644
index f01c724..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CLI.java
+++ /dev/null
@@ -1,63 +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.cli;
-
-import java.io.IOException;
-
-import jline.ConsoleReader;
-
-public class CLI {
-    private static final String HYRACKS_PROMPT = "hyracks> ";
-    private static final String HYRAX_CONTINUE_PROMPT = "> ";
-    private final ConsoleReader reader;
-    private final Session session;
-
-    public CLI(String[] args) throws IOException {
-        reader = new ConsoleReader();
-        session = new Session();
-    }
-
-    public void run() throws IOException {
-        boolean eof = false;
-        while (true) {
-            String prompt = HYRACKS_PROMPT;
-            StringBuilder command = new StringBuilder();
-            while (true) {
-                String line = reader.readLine(prompt);
-                if (line == null) {
-                    eof = true;
-                    break;
-                }
-                prompt = HYRAX_CONTINUE_PROMPT;
-                line = line.trim();
-                command.append(line);
-                if ("".equals(line)) {
-                    break;
-                }
-                if (line.endsWith(";")) {
-                    break;
-                }
-            }
-            if (eof) {
-                break;
-            }
-            try {
-                CommandExecutor.execute(session, command.toString());
-            } catch (Exception e) {
-                e.printStackTrace();
-            }
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CommandExecutor.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CommandExecutor.java
deleted file mode 100644
index 6bdd136..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/CommandExecutor.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.cli;
-
-import java.io.StringReader;
-import java.util.List;
-
-import edu.uci.ics.hyracks.cli.commands.Command;
-
-public class CommandExecutor {
-    public static void execute(Session session, String command) throws Exception {
-        CLIParser parser = new CLIParser(new StringReader(command));
-        List<Command> cmds = parser.Commands();
-        for (Command cmd : cmds) {
-            cmd.run(session);
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Session.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Session.java
deleted file mode 100644
index f15b1bb..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Session.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.cli;
-
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-
-public class Session {
-    private IHyracksClientConnection connection;
-
-    public void setConnection(IHyracksClientConnection connection) {
-        this.connection = connection;
-    }
-
-    public IHyracksClientConnection getConnection() {
-        return connection;
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/Command.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/Command.java
deleted file mode 100644
index ee5849b..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/Command.java
+++ /dev/null
@@ -1,7 +0,0 @@
-package edu.uci.ics.hyracks.cli.commands;
-
-import edu.uci.ics.hyracks.cli.Session;
-
-public abstract class Command {
-    public abstract void run(Session session) throws Exception;
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java
deleted file mode 100644
index 168280b..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/ConnectCommand.java
+++ /dev/null
@@ -1,28 +0,0 @@
-package edu.uci.ics.hyracks.cli.commands;
-
-import edu.uci.ics.hyracks.api.client.HyracksConnection;
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.cli.Session;
-
-public class ConnectCommand extends Command {
-    private String host;
-
-    private int port;
-
-    public ConnectCommand(String hostPortStr) {
-        int idx = hostPortStr.indexOf(':');
-        host = hostPortStr;
-        port = 1098;
-        if (idx != -1) {
-            host = hostPortStr.substring(0, idx);
-            port = Integer.valueOf(hostPortStr.substring(idx + 1));
-        }
-    }
-
-    @Override
-    public void run(Session session) throws Exception {
-        System.err.println("Connecting to host: " + host + ", port: " + port);
-        IHyracksClientConnection conn = new HyracksConnection(host, port);
-        session.setConnection(conn);
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/CreateApplicationCommand.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/CreateApplicationCommand.java
deleted file mode 100644
index 7b35ee5..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/CreateApplicationCommand.java
+++ /dev/null
@@ -1,27 +0,0 @@
-package edu.uci.ics.hyracks.cli.commands;
-
-import java.io.File;
-
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.cli.Session;
-
-public class CreateApplicationCommand extends Command {
-    private String appName;
-
-    private File harFile;
-
-    public CreateApplicationCommand(String appName, File harFile) {
-        this.appName = appName;
-        this.harFile = harFile;
-    }
-
-    @Override
-    public void run(Session session) throws Exception {
-        IHyracksClientConnection hcc = session.getConnection();
-        if (hcc == null) {
-            throw new RuntimeException("Not connected to Hyracks Cluster Controller");
-        }
-        System.err.println("Creating application: " + appName + " with har: " + harFile.getAbsolutePath());
-        hcc.createApplication(appName, harFile);
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DestroyApplicationCommand.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DestroyApplicationCommand.java
deleted file mode 100644
index 03aed1e..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DestroyApplicationCommand.java
+++ /dev/null
@@ -1,22 +0,0 @@
-package edu.uci.ics.hyracks.cli.commands;
-
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.cli.Session;
-
-public class DestroyApplicationCommand extends Command {
-    private String appName;
-
-    public DestroyApplicationCommand(String appName) {
-        this.appName = appName;
-    }
-
-    @Override
-    public void run(Session session) throws Exception {
-        IHyracksClientConnection hcc = session.getConnection();
-        if (hcc == null) {
-            throw new RuntimeException("Not connected to Hyracks Cluster Controller");
-        }
-        System.err.println("Destroying application: " + appName);
-        hcc.destroyApplication(appName);
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DisconnectCommand.java b/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DisconnectCommand.java
deleted file mode 100644
index facf0f1..0000000
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/commands/DisconnectCommand.java
+++ /dev/null
@@ -1,11 +0,0 @@
-package edu.uci.ics.hyracks.cli.commands;
-
-import edu.uci.ics.hyracks.cli.Session;
-
-public class DisconnectCommand extends Command {
-    @Override
-    public void run(Session session) throws Exception {
-        System.err.println("Disconnecting...");
-        session.setConnection(null);
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/src/main/javacc/cli.jj b/hyracks/hyracks-cli/src/main/javacc/cli.jj
deleted file mode 100644
index 0f386cb..0000000
--- a/hyracks/hyracks-cli/src/main/javacc/cli.jj
+++ /dev/null
@@ -1,131 +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.
- */
-options {
-    STATIC = false;
-    IGNORE_CASE = true;
-}
-
-PARSER_BEGIN(CLIParser)
-
-package edu.uci.ics.hyracks.cli;
-
-import java.io.*;
-import java.util.*;
-import edu.uci.ics.hyracks.cli.commands.*;
-
-class CLIParser {
-    private String unquote(String s) {
-        return s.substring(1, s.length() - 1);
-    }
-}
-
-PARSER_END(CLIParser)
-
-List<Command> Commands():
-{
-    List<Command> result = new ArrayList<Command>();
-    Command cmd = null;
-} {
-    (
-        (
-            cmd = ConnectCommand()
-            | cmd = DisconnectCommand()
-            | cmd = CreateApplicationCommand()
-            | cmd = DestroyApplicationCommand()
-        ) ";" {
-            result.add(cmd);
-        }
-    )* <EOF> {
-        return result;
-    }
-}
-
-protected Command ConnectCommand():
-{
-    Token t;
-} {
-    "connect" "to" t = <StringLiteral> {
-        return new ConnectCommand(unquote(t.image));
-    }
-}
-
-protected Command CreateApplicationCommand():
-{
-    Token a;
-    Token f;
-} {
-    "create" "application" a = <Identifier> f = <StringLiteral> {
-        return new CreateApplicationCommand(a.image, new File(unquote(f.image)));
-    }
-}
-
-protected Command DestroyApplicationCommand():
-{
-    Token a;
-} {
-    "destroy" "application" a = <Identifier> {
-        return new DestroyApplicationCommand(a.image);
-    }
-}
-
-protected Command DisconnectCommand():
-{
-    Token t;
-} {
-    "disconnect" {
-        return new DisconnectCommand();
-    }
-}
-
-<DEFAULT>
-TOKEN : {
-    <StringLiteral: (("\"" (~["\"", "\n"])* "\"") | ("'" (~["'", "\n"])* "'"))>
-    | <Identifier: <Letter> (<Letter> | <Digit> | <Extender>)*>
-}
-
-SPECIAL_TOKEN :
-{
- < WhitespaceChar : ["\t", "\r", "\n", " "] >
-}
-
-TOKEN :
-{
- < #Letter : (<BaseChar> | <Ideographic>) >
-}
-
-TOKEN :
-{
- < #BaseChar : ["\u0041" - "\u005a", "\u0061" - "\u007a", "\u00c0" - "\u00d6", "\u00d8" - "\u00f6", "\u00f8" - "\u00ff", "\u0100" - "\u0131", "\u0134" - "\u013e", "\u0141" - "\u0148", "\u014a" - "\u017e", "\u0180" - "\u01c3", "\u01cd" - "\u01f0", "\u01f4" - "\u01f5", "\u01fa" - "\u0217", "\u0250" - "\u02a8", "\u02bb" - "\u02c1", "\u0386", "\u0388" - "\u038a", "\u038c", "\u038e" - "\u03a1", "\u03a3" - "\u03ce", "\u03d0" - "\u03d6", "\u03da", "\u03dc", "\u03de", "\u03e0", "\u03e2" - "\u03f3", "\u0401" - "\u040c", "\u040e" - "\u044f", "\u0451" - "\u045c", "\u045e" - "\u0481", "\u0490" - "\u04c4", "\u04c7" - "\u04c8", "\u04cb" - "\u04cc", "\u04d0" - "\u04eb", "\u04ee" - "\u04f5", "\u04f8" - "\u04f9", "\u0531" - "\u0556", "\u0559", "\u0561" - "\u0586", "\u05d0" - "\u05ea", "\u05f0" - "\u05f2", "\u0621" - "\u063a", "\u0641" - "\u064a", "\u0671" - "\u06b7", "\u06ba" - "\u06be", "\u06c0" - "\u06ce", "\u06d0" - "\u06d3", "\u06d5", "\u06e5" - "\u06e6", "\u0905" - "\u0939", "\u093d", "\u0958" - "\u0961", "\u0985" - "\u098c", "\u098f" - "\u0990", "\u0993" - "\u09a8", "\u09aa" - "\u09b0", "\u09b2", "\u09b6" - "\u09b9", "\u09dc" - "\u09dd", "\u09df" - "\u09e1", "\u09f0" - "\u09f1", "\u0a05" - "\u0a0a", "\u0a0f" - "\u0a10", "\u0a13" - "\u0a28", "\u0a2a" - "\u0a30", "\u0a32" - "\u0a33", "\u0a35" - "\u0a36", "\u0a38" - "\u0a39", "\u0a59" - "\u0a5c", "\u0a5e", "\u0a72" - "\u0a74", "\u0a85" - "\u0a8b", "\u0a8d", "\u0a8f" - "\u0a91", "\u0a93" - "\u0aa8", "\u0aaa" - "\u0ab0", "\u0ab2" - "\u0ab3", "\u0ab5" - "\u0ab9", "\u0abd", "\u0ae0", "\u0b05" - "\u0b0c", "\u0b0f" - "\u0b10", "\u0b13" - "\u0b28", "\u0b2a" - "\u0b30", "\u0b32" - "\u0b33", "\u0b36" - "\u0b39", "\u0b3d", "\u0b5c" - "\u0b5d", "\u0b5f" - "\u0b61", "\u0b85" - "\u0b8a", "\u0b8e" - "\u0b90", "\u0b92" - "\u0b95", "\u0b99" - "\u0b9a", "\u0b9c", "\u0b9e" - "\u0b9f", "\u0ba3" - "\u0ba4", "\u0ba8" - "\u0baa", "\u0bae" - "\u0bb5", "\u0bb7" - "\u0bb9", "\u0c05" - "\u0c0c", "\u0c0e" - "\u0c10", "\u0c12" - "\u0c28", "\u0c2a" - "\u0c33", "\u0c35" - "\u0c39", "\u0c60" - "\u0c61", "\u0c85" - "\u0c8c", "\u0c8e" - "\u0c90", "\u0c92" - "\u0ca8", "\u0caa" - "\u0cb3", "\u0cb5" - "\u0cb9", "\u0cde", "\u0ce0" - "\u0ce1", "\u0d05" - "\u0d0c", "\u0d0e" - "\u0d10", "\u0d12" - "\u0d28", "\u0d2a" - "\u0d39", "\u0d60" - "\u0d61", "\u0e01" - "\u0e2e", "\u0e30", "\u0e32" - "\u0e33", "\u0e40" - "\u0e45", "\u0e81" - "\u0e82", "\u0e84", "\u0e87" - "\u0e88", "\u0e8a", "\u0e8d", "\u0e94" - "\u0e97", "\u0e99" - "\u0e9f", "\u0ea1" - "\u0ea3", "\u0ea5", "\u0ea7", "\u0eaa" - "\u0eab", "\u0ead" - "\u0eae", "\u0eb0", "\u0eb2" - "\u0eb3", "\u0ebd", "\u0ec0" - "\u0ec4", "\u0f40" - "\u0f47", "\u0f49" - "\u0f69", "\u10a0" - "\u10c5", "\u10d0" - "\u10f6", "\u1100", "\u1102" - "\u1103", "\u1105" - "\u1107", "\u1109", "\u110b" - "\u110c", "\u110e" - "\u1112", "\u113c", "\u113e", "\u1140", "\u114c", "\u114e", "\u1150", "\u1154" - "\u1155", "\u1159", "\u115f" - "\u1161", "\u1163", "\u1165", "\u1167", "\u1169", "\u116d" - "\u116e", "\u1172" - "\u1173", "\u1175", "\u119e", "\u11a8", "\u11ab", "\u11ae" - "\u11af", "\u11b7" - "\u11b8", "\u11ba", "\u11bc" - "\u11c2", "\u11eb", "\u11f0", "\u11f9", "\u1e00" - "\u1e9b", "\u1ea0" - "\u1ef9", "\u1f00" - "\u1f15", "\u1f18" - "\u1f1d", "\u1f20" - "\u1f45", "\u1f48" - "\u1f4d", "\u1f50" - "\u1f57", "\u1f59", "\u1f5b", "\u1f5d", "\u1f5f" - "\u1f7d", "\u1f80" - "\u1fb4", "\u1fb6" - "\u1fbc", "\u1fbe", "\u1fc2" - "\u1fc4", "\u1fc6" - "\u1fcc", "\u1fd0" - "\u1fd3", "\u1fd6" - "\u1fdb", "\u1fe0" - "\u1fec", "\u1ff2" - "\u1ff4", "\u1ff6" - "\u1ffc", "\u2126", "\u212a" - "\u212b", "\u212e", "\u2180" - "\u2182", "\u3041" - "\u3094", "\u30a1" - "\u30fa", "\u3105" - "\u312c", "\uac00" - "\ud7a3"] >
-}
-
-TOKEN :
-{
- < #Ideographic : ["\u4e00" - "\u9fa5", "\u3007", "\u3021" - "\u3029"] >
-}
-
-TOKEN :
-{
- < #CombiningChar : ["\u0300" - "\u0345", "\u0360" - "\u0361", "\u0483" - "\u0486", "\u0591" - "\u05a1", "\u05a3" - "\u05b9", "\u05bb" - "\u05bd", "\u05bf", "\u05c1" - "\u05c2", "\u05c4", "\u064b" - "\u0652", "\u0670", "\u06d6" - "\u06dc", "\u06dd" - "\u06df", "\u06e0" - "\u06e4", "\u06e7" - "\u06e8", "\u06ea" - "\u06ed", "\u0901" - "\u0903", "\u093c", "\u093e" - "\u094c", "\u094d", "\u0951" - "\u0954", "\u0962" - "\u0963", "\u0981" - "\u0983", "\u09bc", "\u09be", "\u09bf", "\u09c0" - "\u09c4", "\u09c7" - "\u09c8", "\u09cb" - "\u09cd", "\u09d7", "\u09e2" - "\u09e3", "\u0a02", "\u0a3c", "\u0a3e", "\u0a3f", "\u0a40" - "\u0a42", "\u0a47" - "\u0a48", "\u0a4b" - "\u0a4d", "\u0a70" - "\u0a71", "\u0a81" - "\u0a83", "\u0abc", "\u0abe" - "\u0ac5", "\u0ac7" - "\u0ac9", "\u0acb" - "\u0acd", "\u0b01" - "\u0b03", "\u0b3c", "\u0b3e" - "\u0b43", "\u0b47" - "\u0b48", "\u0b4b" - "\u0b4d", "\u0b56" - "\u0b57", "\u0b82" - "\u0b83", "\u0bbe" - "\u0bc2", "\u0bc6" - "\u0bc8", "\u0bca" - "\u0bcd", "\u0bd7", "\u0c01" - "\u0c03", "\u0c3e" - "\u0c44", "\u0c46" - "\u0c48", "\u0c4a" - "\u0c4d", "\u0c55" - "\u0c56", "\u0c82" - "\u0c83", "\u0cbe" - "\u0cc4", "\u0cc6" - "\u0cc8", "\u0cca" - "\u0ccd", "\u0cd5" - "\u0cd6", "\u0d02" - "\u0d03", "\u0d3e" - "\u0d43", "\u0d46" - "\u0d48", "\u0d4a" - "\u0d4d", "\u0d57", "\u0e31", "\u0e34" - "\u0e3a", "\u0e47" - "\u0e4e", "\u0eb1", "\u0eb4" - "\u0eb9", "\u0ebb" - "\u0ebc", "\u0ec8" - "\u0ecd", "\u0f18" - "\u0f19", "\u0f35", "\u0f37", "\u0f39", "\u0f3e", "\u0f3f", "\u0f71" - "\u0f84", "\u0f86" - "\u0f8b", "\u0f90" - "\u0f95", "\u0f97", "\u0f99" - "\u0fad", "\u0fb1" - "\u0fb7", "\u0fb9", "\u20d0" - "\u20dc", "\u20e1", "\u302a" - "\u302f", "\u3099", "\u309a"] >
-}
-
-TOKEN :
-{
- < #Digit : ["\u0030" - "\u0039", "\u0660" - "\u0669", "\u06f0" - "\u06f9", "\u0966" - "\u096f", "\u09e6" - "\u09ef", "\u0a66" - "\u0a6f", "\u0ae6" - "\u0aef", "\u0b66" - "\u0b6f", "\u0be7" - "\u0bef", "\u0c66" - "\u0c6f", "\u0ce6" - "\u0cef", "\u0d66" - "\u0d6f", "\u0e50" - "\u0e59", "\u0ed0" - "\u0ed9", "\u0f20" - "\u0f29"] >
-}
-
-TOKEN :
-{
- < #Extender : ["\u00b7", "\u02d0", "\u02d1", "\u0387", "\u0640", "\u0e46", "\u0ec6", "\u3005", "\u3031" - "\u3035", "\u309d" - "\u309e", "\u30fc" - "\u30fe"] >
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-client/pom.xml b/hyracks/hyracks-client/pom.xml
new file mode 100644
index 0000000..854a009
--- /dev/null
+++ b/hyracks/hyracks-client/pom.xml
@@ -0,0 +1,46 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>hyracks-client</artifactId>
+  <name>hyracks-client</name>
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+ <dependency>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-api</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-net</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-comm</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+  <dependency>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-dataflow-common</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+  </dependencies>
+</project>
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
new file mode 100644
index 0000000..8be4a8c
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.client.dataset;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+
+public class DatasetClientContext implements IHyracksCommonContext {
+    private final int frameSize;
+
+    public DatasetClientContext(int frameSize) {
+        this.frameSize = frameSize;
+    }
+
+    @Override
+    public int getFrameSize() {
+        return frameSize;
+    }
+
+    @Override
+    public IIOManager getIOManager() {
+        return null;
+    }
+
+    @Override
+    public ByteBuffer allocateFrame() {
+        return ByteBuffer.allocate(frameSize);
+    }
+
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDataset.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDataset.java
new file mode 100644
index 0000000..6866e46
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDataset.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.client.dataset;
+
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.client.net.ClientNetworkManager;
+
+public class HyracksDataset implements IHyracksDataset {
+    private final IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection;
+
+    private final ClientNetworkManager netManager;
+
+    private final DatasetClientContext datasetClientCtx;
+
+    public HyracksDataset(IHyracksClientConnection hcc, int frameSize, int nReaders) throws Exception {
+        NetworkAddress ddsAddress = hcc.getDatasetDirectoryServiceInfo();
+        datasetDirectoryServiceConnection = new HyracksDatasetDirectoryServiceConnection(new String(
+                ddsAddress.getIpAddress()), ddsAddress.getPort());
+
+        netManager = new ClientNetworkManager(nReaders);
+        netManager.start();
+
+        datasetClientCtx = new DatasetClientContext(frameSize);
+    }
+
+    @Override
+    public IHyracksDatasetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
+        IHyracksDatasetReader reader = null;
+        try {
+            reader = new HyracksDatasetReader(datasetDirectoryServiceConnection, netManager, datasetClientCtx, jobId,
+                    resultSetId);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+        return reader;
+    }
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java
new file mode 100644
index 0000000..095fd7d
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.client.dataset;
+
+import java.net.InetSocketAddress;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceInterface;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
+import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
+import edu.uci.ics.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
+
+//TODO(madhusudancs): Should this implementation be moved to edu.uci.ics.hyracks.client?
+public class HyracksDatasetDirectoryServiceConnection implements IHyracksDatasetDirectoryServiceConnection {
+    private final IPCSystem ipc;
+    private final IHyracksDatasetDirectoryServiceInterface ddsi;
+
+    public HyracksDatasetDirectoryServiceConnection(String ddsHost, int ddsPort) throws Exception {
+        RPCInterface rpci = new RPCInterface();
+        ipc = new IPCSystem(new InetSocketAddress(0), rpci, new JavaSerializationBasedPayloadSerializerDeserializer());
+        ipc.start();
+        IIPCHandle ddsIpchandle = ipc.getHandle(new InetSocketAddress(ddsHost, ddsPort));
+        this.ddsi = new HyracksDatasetDirectoryServiceInterfaceRemoteProxy(ddsIpchandle, rpci);
+    }
+
+    @Override
+    public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception {
+        return ddsi.getDatasetResultStatus(jobId, rsId);
+    }
+
+    @Override
+    public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownRecords) throws Exception {
+        return ddsi.getDatasetResultLocations(jobId, rsId, knownRecords);
+    }
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java
new file mode 100644
index 0000000..47cdf97
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.client.dataset;
+
+import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceInterface;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
+
+//TODO(madhusudancs): Should this implementation be moved to edu.uci.ics.hyracks.client?
+public class HyracksDatasetDirectoryServiceInterfaceRemoteProxy implements IHyracksDatasetDirectoryServiceInterface {
+    private final IIPCHandle ipcHandle;
+
+    private final RPCInterface rpci;
+
+    public HyracksDatasetDirectoryServiceInterfaceRemoteProxy(IIPCHandle ipcHandle, RPCInterface rpci) {
+        this.ipcHandle = ipcHandle;
+        this.rpci = rpci;
+    }
+
+    @Override
+    public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception {
+        HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction gdrlf = new HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction(
+                jobId, rsId);
+        return (Status) rpci.call(ipcHandle, gdrlf);
+    }
+
+    @Override
+    public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownRecords) throws Exception {
+        HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction gdrlf = new HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction(
+                jobId, rsId, knownRecords);
+        return (DatasetDirectoryRecord[]) rpci.call(ipcHandle, gdrlf);
+    }
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
new file mode 100644
index 0000000..78bcf20
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
@@ -0,0 +1,255 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.client.dataset;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IDatasetInputChannelMonitor;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.client.net.ClientNetworkManager;
+import edu.uci.ics.hyracks.comm.channels.DatasetNetworkInputChannel;
+
+// TODO(madhusudancs): Should this implementation be moved to edu.uci.ics.hyracks.client?
+public class HyracksDatasetReader implements IHyracksDatasetReader {
+    private static final Logger LOGGER = Logger.getLogger(HyracksDatasetReader.class.getName());
+
+    private final IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection;
+
+    private final ClientNetworkManager netManager;
+
+    private final DatasetClientContext datasetClientCtx;
+
+    private JobId jobId;
+
+    private ResultSetId resultSetId;
+
+    private DatasetDirectoryRecord[] knownRecords;
+
+    private IDatasetInputChannelMonitor[] monitors;
+
+    private int lastReadPartition;
+
+    private IDatasetInputChannelMonitor lastMonitor;
+
+    private DatasetNetworkInputChannel resultChannel;
+
+    private static int NUM_READ_BUFFERS = 1;
+
+    public HyracksDatasetReader(IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection,
+            ClientNetworkManager netManager, DatasetClientContext datasetClientCtx, JobId jobId, ResultSetId resultSetId)
+            throws Exception {
+        this.datasetDirectoryServiceConnection = datasetDirectoryServiceConnection;
+        this.netManager = netManager;
+        this.datasetClientCtx = datasetClientCtx;
+        this.jobId = jobId;
+        this.resultSetId = resultSetId;
+        knownRecords = null;
+        monitors = null;
+        lastReadPartition = -1;
+        lastMonitor = null;
+        resultChannel = null;
+    }
+
+    @Override
+    public Status getResultStatus() {
+        Status status = null;
+        try {
+            status = datasetDirectoryServiceConnection.getDatasetResultStatus(jobId, resultSetId);
+        } catch (Exception e) {
+            // TODO(madhusudancs): Decide what to do in case of error
+        }
+        return status;
+    }
+
+    @Override
+    public int read(ByteBuffer buffer) throws HyracksDataException {
+        ByteBuffer readBuffer;
+        int readSize = 0;
+
+        if (lastReadPartition == -1) {
+            while (knownRecords == null || knownRecords[0] == null) {
+                try {
+                    knownRecords = datasetDirectoryServiceConnection.getDatasetResultLocations(jobId, resultSetId,
+                            knownRecords);
+                    lastReadPartition = 0;
+                    resultChannel = new DatasetNetworkInputChannel(netManager,
+                            getSocketAddress(knownRecords[lastReadPartition]), jobId, lastReadPartition,
+                            NUM_READ_BUFFERS);
+                    lastMonitor = getMonitor(lastReadPartition);
+                    resultChannel.open(datasetClientCtx);
+                    resultChannel.registerMonitor(lastMonitor);
+                } catch (HyracksException e) {
+                    throw new HyracksDataException(e);
+                } catch (UnknownHostException e) {
+                    throw new HyracksDataException(e);
+                } catch (Exception e) {
+                    // Do nothing here.
+                }
+            }
+        }
+
+        while (readSize <= 0 && !((lastReadPartition == knownRecords.length - 1) && (lastMonitor.eosReached()))) {
+            synchronized (lastMonitor) {
+                while (lastMonitor.getNFramesAvailable() <= 0 && !lastMonitor.eosReached()) {
+                    try {
+                        lastMonitor.wait();
+                    } catch (InterruptedException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+            }
+
+            if (lastMonitor.getNFramesAvailable() <= 0 && lastMonitor.eosReached()) {
+                knownRecords[lastReadPartition].readEOS();
+                if ((lastReadPartition == knownRecords.length - 1)) {
+                    break;
+                } else {
+                    try {
+                        lastReadPartition++;
+                        while (knownRecords[lastReadPartition] == null) {
+                            try {
+                                knownRecords = datasetDirectoryServiceConnection.getDatasetResultLocations(jobId,
+                                        resultSetId, knownRecords);
+                            } catch (Exception e) {
+                                // Do nothing here.
+                            }
+                        }
+
+                        resultChannel = new DatasetNetworkInputChannel(netManager,
+                                getSocketAddress(knownRecords[lastReadPartition]), jobId, lastReadPartition,
+                                NUM_READ_BUFFERS);
+                        lastMonitor = getMonitor(lastReadPartition);
+                        resultChannel.open(datasetClientCtx);
+                        resultChannel.registerMonitor(lastMonitor);
+                    } catch (HyracksException e) {
+                        throw new HyracksDataException(e);
+                    } catch (UnknownHostException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+            } else {
+                readBuffer = resultChannel.getNextBuffer();
+                lastMonitor.notifyFrameRead();
+                if (readBuffer != null) {
+                    buffer.put(readBuffer);
+                    buffer.flip();
+                    readSize = buffer.limit();
+                    resultChannel.recycleBuffer(readBuffer);
+                }
+            }
+        }
+
+        return readSize;
+    }
+
+    private boolean nullExists(DatasetDirectoryRecord[] locations) {
+        if (locations == null) {
+            return true;
+        }
+        for (int i = 0; i < locations.length; i++) {
+            if (locations[i] == null) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private SocketAddress getSocketAddress(DatasetDirectoryRecord addr) throws UnknownHostException {
+        NetworkAddress netAddr = addr.getNetworkAddress();
+        return new InetSocketAddress(InetAddress.getByAddress(netAddr.getIpAddress()), netAddr.getPort());
+    }
+
+    private IDatasetInputChannelMonitor getMonitor(int partition) throws HyracksException {
+        if (knownRecords == null || knownRecords[partition] == null) {
+            throw new HyracksException("Accessing monitors before the obtaining the corresponding addresses.");
+        }
+        if (monitors == null) {
+            monitors = new DatasetInputChannelMonitor[knownRecords.length];
+        }
+        if (monitors[partition] == null) {
+            monitors[partition] = new DatasetInputChannelMonitor();
+        }
+        return monitors[partition];
+    }
+
+    private class DatasetInputChannelMonitor implements IDatasetInputChannelMonitor {
+        private final AtomicInteger nAvailableFrames;
+
+        private final AtomicBoolean eos;
+
+        private final AtomicBoolean failed;
+
+        public DatasetInputChannelMonitor() {
+            nAvailableFrames = new AtomicInteger(0);
+            eos = new AtomicBoolean(false);
+            failed = new AtomicBoolean(false);
+        }
+
+        @Override
+        public synchronized void notifyFailure(IInputChannel channel) {
+            failed.set(true);
+            notifyAll();
+        }
+
+        @Override
+        public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
+            nAvailableFrames.addAndGet(nFrames);
+            notifyAll();
+        }
+
+        @Override
+        public synchronized void notifyEndOfStream(IInputChannel channel) {
+            eos.set(true);
+            notifyAll();
+        }
+
+        @Override
+        public synchronized boolean eosReached() {
+            return eos.get();
+        }
+
+        @Override
+        public synchronized boolean failed() {
+            return failed.get();
+        }
+
+        @Override
+        public synchronized int getNFramesAvailable() {
+            return nAvailableFrames.get();
+        }
+
+        @Override
+        public synchronized void notifyFrameRead() {
+            nAvailableFrames.decrementAndGet();
+        }
+
+    }
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/net/ClientNetworkManager.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/net/ClientNetworkManager.java
new file mode 100644
index 0000000..7aef8b9
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/net/ClientNetworkManager.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.client.net;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+
+import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
+
+public class ClientNetworkManager implements IChannelConnectionFactory {
+    private static final int MAX_CONNECTION_ATTEMPTS = 5;
+
+    private final MuxDemux md;
+
+    public ClientNetworkManager(int nThreads) throws IOException {
+        /* This is a connect only socket and does not listen to any incoming connections, so pass null to
+         * localAddress and listener.
+         */
+        md = new MuxDemux(null, null, nThreads, MAX_CONNECTION_ATTEMPTS);
+    }
+
+    public void start() throws IOException {
+        md.start();
+    }
+
+    public void stop() {
+
+    }
+
+    public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException {
+        MultiplexedConnection mConn = md.connect((InetSocketAddress) remoteAddress);
+        return mConn.openChannel();
+    }
+
+    public MuxDemuxPerformanceCounters getPerformanceCounters() {
+        return md.getPerformanceCounters();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-comm/pom.xml b/hyracks/hyracks-comm/pom.xml
new file mode 100644
index 0000000..c3583699
--- /dev/null
+++ b/hyracks/hyracks-comm/pom.xml
@@ -0,0 +1,36 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>hyracks-comm</artifactId>
+  <name>hyracks-comm</name>
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+ <dependency>
+ 	<groupId>edu.uci.ics.hyracks</groupId>
+ 	<artifactId>hyracks-api</artifactId>
+ 	<version>0.2.3-SNAPSHOT</version>
+ </dependency>
+  <dependency>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-net</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+  </dependencies>
+</project>
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
similarity index 72%
copy from hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
copy to hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
index 1d5af84..fac2949 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.nc.net;
+package edu.uci.ics.hyracks.comm.channels;
 
 import java.net.SocketAddress;
 import java.nio.ByteBuffer;
@@ -23,21 +23,25 @@
 
 import edu.uci.ics.hyracks.api.channels.IInputChannel;
 import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
 import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
 import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
 
-public class NetworkInputChannel implements IInputChannel {
-    private static final Logger LOGGER = Logger.getLogger(NetworkInputChannel.class.getName());
+public class DatasetNetworkInputChannel implements IInputChannel {
+    private static final Logger LOGGER = Logger.getLogger(DatasetNetworkInputChannel.class.getName());
 
-    private final NetworkManager netManager;
+    static final int INITIAL_MESSAGE_SIZE = 20;
+
+    private final IChannelConnectionFactory netManager;
 
     private final SocketAddress remoteAddress;
 
-    private final PartitionId partitionId;
+    private final JobId jobId;
+
+    private final int partition;
 
     private final Queue<ByteBuffer> fullQueue;
 
@@ -49,11 +53,12 @@
 
     private Object attachment;
 
-    public NetworkInputChannel(NetworkManager netManager, SocketAddress remoteAddress, PartitionId partitionId,
-            int nBuffers) {
+    public DatasetNetworkInputChannel(IChannelConnectionFactory netManager, SocketAddress remoteAddress, JobId jobId,
+            int partition, int nBuffers) {
         this.netManager = netManager;
         this.remoteAddress = remoteAddress;
-        this.partitionId = partitionId;
+        this.jobId = jobId;
+        this.partition = partition;
         fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
         this.nBuffers = nBuffers;
     }
@@ -85,7 +90,7 @@
     }
 
     @Override
-    public void open(IHyracksTaskContext ctx) throws HyracksDataException {
+    public void open(IHyracksCommonContext ctx) throws HyracksDataException {
         try {
             ccb = netManager.connect(remoteAddress);
         } catch (Exception e) {
@@ -96,14 +101,13 @@
         for (int i = 0; i < nBuffers; ++i) {
             ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
         }
-        ByteBuffer writeBuffer = ByteBuffer.allocate(NetworkManager.INITIAL_MESSAGE_SIZE);
-        writeBuffer.putLong(partitionId.getJobId().getId());
-        writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
-        writeBuffer.putInt(partitionId.getSenderIndex());
-        writeBuffer.putInt(partitionId.getReceiverIndex());
+        ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
+        writeBuffer.putLong(jobId.getId());
+        writeBuffer.putInt(partition);
         writeBuffer.flip();
         if (LOGGER.isLoggable(Level.FINE)) {
-            LOGGER.fine("Sending partition request: " + partitionId + " on channel: " + ccb);
+            LOGGER.fine("Sending partition request for JobId: " + jobId + " partition: " + partition + " on channel: "
+                    + ccb);
         }
         ccb.getWriteInterface().getFullBufferAcceptor().accept(writeBuffer);
         ccb.getWriteInterface().getFullBufferAcceptor().close();
@@ -118,17 +122,17 @@
         @Override
         public void accept(ByteBuffer buffer) {
             fullQueue.add(buffer);
-            monitor.notifyDataAvailability(NetworkInputChannel.this, 1);
+            monitor.notifyDataAvailability(DatasetNetworkInputChannel.this, 1);
         }
 
         @Override
         public void close() {
-            monitor.notifyEndOfStream(NetworkInputChannel.this);
+            monitor.notifyEndOfStream(DatasetNetworkInputChannel.this);
         }
 
         @Override
         public void error(int ecode) {
-            monitor.notifyFailure(NetworkInputChannel.this);
+            monitor.notifyFailure(DatasetNetworkInputChannel.this);
         }
     }
 
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/IChannelConnectionFactory.java
similarity index 63%
copy from hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
copy to hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/IChannelConnectionFactory.java
index 5f7ef0b..33179ba 100644
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/IChannelConnectionFactory.java
@@ -12,11 +12,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.cli;
+package edu.uci.ics.hyracks.comm.channels;
 
-public class Main {
-    public static void main(String[] args) throws Exception {
-        CLI cli = new CLI(args);
-        cli.run();
-    }
-}
\ No newline at end of file
+import java.net.SocketAddress;
+
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+
+public interface IChannelConnectionFactory {
+    public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException;
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
similarity index 89%
rename from hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
rename to hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
index 1d5af84..aa37b16 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.nc.net;
+package edu.uci.ics.hyracks.comm.channels;
 
 import java.net.SocketAddress;
 import java.nio.ByteBuffer;
@@ -23,7 +23,7 @@
 
 import edu.uci.ics.hyracks.api.channels.IInputChannel;
 import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
 import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
@@ -33,7 +33,9 @@
 public class NetworkInputChannel implements IInputChannel {
     private static final Logger LOGGER = Logger.getLogger(NetworkInputChannel.class.getName());
 
-    private final NetworkManager netManager;
+    static final int INITIAL_MESSAGE_SIZE = 20;
+
+    private final IChannelConnectionFactory netManager;
 
     private final SocketAddress remoteAddress;
 
@@ -49,8 +51,8 @@
 
     private Object attachment;
 
-    public NetworkInputChannel(NetworkManager netManager, SocketAddress remoteAddress, PartitionId partitionId,
-            int nBuffers) {
+    public NetworkInputChannel(IChannelConnectionFactory netManager, SocketAddress remoteAddress,
+            PartitionId partitionId, int nBuffers) {
         this.netManager = netManager;
         this.remoteAddress = remoteAddress;
         this.partitionId = partitionId;
@@ -85,7 +87,7 @@
     }
 
     @Override
-    public void open(IHyracksTaskContext ctx) throws HyracksDataException {
+    public void open(IHyracksCommonContext ctx) throws HyracksDataException {
         try {
             ccb = netManager.connect(remoteAddress);
         } catch (Exception e) {
@@ -96,7 +98,7 @@
         for (int i = 0; i < nBuffers; ++i) {
             ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
         }
-        ByteBuffer writeBuffer = ByteBuffer.allocate(NetworkManager.INITIAL_MESSAGE_SIZE);
+        ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
         writeBuffer.putLong(partitionId.getJobId().getId());
         writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
         writeBuffer.putInt(partitionId.getSenderIndex());
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
similarity index 92%
rename from hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
rename to hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
index 9024e18..812a2de 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
@@ -12,14 +12,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.nc.net;
+package edu.uci.ics.hyracks.comm.channels;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayDeque;
 import java.util.Deque;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
 import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
@@ -40,9 +39,9 @@
         ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
     }
 
-    public void setTaskContext(IHyracksTaskContext ctx) {
+    public void setFrameSize(int frameSize) {
         for (int i = 0; i < nBuffers; ++i) {
-            emptyStack.push(ByteBuffer.allocateDirect(ctx.getFrameSize()));
+            emptyStack.push(ByteBuffer.allocateDirect(frameSize));
         }
     }
 
@@ -87,7 +86,7 @@
         ccb.getWriteInterface().getFullBufferAcceptor().close();
     }
 
-    void abort() {
+    public void abort() {
         ccb.getWriteInterface().getFullBufferAcceptor().error(1);
         synchronized (NetworkOutputChannel.this) {
             aborted = true;
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index 5a33891..a8055c9 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -18,7 +18,6 @@
 import java.io.FileReader;
 import java.net.InetSocketAddress;
 import java.util.HashMap;
-import java.util.Hashtable;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Set;
@@ -32,33 +31,41 @@
 
 import org.xml.sax.InputSource;
 
+import edu.uci.ics.hyracks.api.application.ICCApplicationEntryPoint;
 import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
 import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
 import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.context.ICCContext;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.api.topology.ClusterTopology;
 import edu.uci.ics.hyracks.api.topology.TopologyDefinitionParser;
 import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.cc.dataset.DatasetDirectoryService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
 import edu.uci.ics.hyracks.control.cc.web.WebServer;
-import edu.uci.ics.hyracks.control.cc.work.ApplicationCreateWork;
-import edu.uci.ics.hyracks.control.cc.work.ApplicationDestroyWork;
 import edu.uci.ics.hyracks.control.cc.work.ApplicationMessageWork;
-import edu.uci.ics.hyracks.control.cc.work.ApplicationStartWork;
-import edu.uci.ics.hyracks.control.cc.work.ApplicationStateChangeWork;
+import edu.uci.ics.hyracks.control.cc.work.GetDatasetDirectoryServiceInfoWork;
 import edu.uci.ics.hyracks.control.cc.work.GetIpAddressNodeNameMapWork;
 import edu.uci.ics.hyracks.control.cc.work.GetJobStatusWork;
 import edu.uci.ics.hyracks.control.cc.work.GetNodeControllersInfoWork;
+import edu.uci.ics.hyracks.control.cc.work.GetResultPartitionLocationsWork;
+import edu.uci.ics.hyracks.control.cc.work.GetResultStatusWork;
 import edu.uci.ics.hyracks.control.cc.work.JobStartWork;
 import edu.uci.ics.hyracks.control.cc.work.JobletCleanupNotificationWork;
 import edu.uci.ics.hyracks.control.cc.work.NodeHeartbeatWork;
 import edu.uci.ics.hyracks.control.cc.work.RegisterNodeWork;
 import edu.uci.ics.hyracks.control.cc.work.RegisterPartitionAvailibilityWork;
 import edu.uci.ics.hyracks.control.cc.work.RegisterPartitionRequestWork;
+import edu.uci.ics.hyracks.control.cc.work.RegisterResultPartitionLocationWork;
 import edu.uci.ics.hyracks.control.cc.work.RemoveDeadNodesWork;
 import edu.uci.ics.hyracks.control.cc.work.ReportProfilesWork;
+import edu.uci.ics.hyracks.control.cc.work.ReportResultPartitionFailureWork;
+import edu.uci.ics.hyracks.control.cc.work.ReportResultPartitionWriteCompletionWork;
 import edu.uci.ics.hyracks.control.cc.work.TaskCompleteWork;
 import edu.uci.ics.hyracks.control.cc.work.TaskFailureWork;
 import edu.uci.ics.hyracks.control.cc.work.UnregisterNodeWork;
@@ -93,14 +100,14 @@
 
     private final Map<String, Set<String>> ipAddressNodeNameMap;
 
-    private final Map<String, CCApplicationContext> applications;
-
     private final ServerContext serverCtx;
 
     private final WebServer webServer;
 
     private ClusterControllerInfo info;
 
+    private CCApplicationContext appCtx;
+
     private final Map<JobId, JobRun> activeRunMap;
 
     private final Map<JobId, JobRun> runMapArchive;
@@ -115,6 +122,8 @@
 
     private final DeadNodeSweeper sweeper;
 
+    private final IDatasetDirectoryService datasetDirectoryService;
+
     private long jobCounter;
 
     public ClusterControllerService(final CCConfig ccConfig) throws Exception {
@@ -123,7 +132,6 @@
         jobLog = new LogFile(jobLogFolder);
         nodeRegistry = new LinkedHashMap<String, NodeControllerState>();
         ipAddressNodeNameMap = new HashMap<String, Set<String>>();
-        applications = new Hashtable<String, CCApplicationContext>();
         serverCtx = new ServerContext(ServerContext.ServerType.CLUSTER_CONTROLLER, new File(ccConfig.ccRoot));
         executor = Executors.newCachedThreadPool();
         IIPCI ccIPCI = new ClusterControllerIPCI();
@@ -162,6 +170,7 @@
             }
         };
         sweeper = new DeadNodeSweeper();
+        datasetDirectoryService = new DatasetDirectoryService();
         jobCounter = 0;
     }
 
@@ -190,9 +199,22 @@
                 webServer.getListeningPort());
         timer.schedule(sweeper, 0, ccConfig.heartbeatPeriod);
         jobLog.open();
+        startApplication();
         LOGGER.log(Level.INFO, "Started ClusterControllerService");
     }
 
+    private void startApplication() throws Exception {
+        appCtx = new CCApplicationContext(serverCtx, ccContext);
+        String className = ccConfig.appCCMainClass;
+        if (className != null) {
+            Class<?> c = Class.forName(className);
+            ICCApplicationEntryPoint aep = (ICCApplicationEntryPoint) c.newInstance();
+            String[] args = ccConfig.appArgs == null ? null : ccConfig.appArgs.toArray(new String[ccConfig.appArgs
+                    .size()]);
+            aep.start(appCtx, args);
+        }
+    }
+
     @Override
     public void stop() throws Exception {
         LOGGER.log(Level.INFO, "Stopping ClusterControllerService");
@@ -212,10 +234,6 @@
         return ccContext;
     }
 
-    public Map<String, CCApplicationContext> getApplicationMap() {
-        return applications;
-    }
-
     public Map<JobId, JobRun> getActiveRunMap() {
         return activeRunMap;
     }
@@ -248,6 +266,10 @@
         return ccConfig;
     }
 
+    public CCApplicationContext getApplicationContext() {
+        return appCtx;
+    }
+
     private JobId createJobId() {
         return new JobId(jobCounter++);
     }
@@ -264,6 +286,10 @@
         return clusterIPC;
     }
 
+    public NetworkAddress getDatasetDirectoryServiceInfo() {
+        return new NetworkAddress(ccConfig.clientNetIpAddress.getBytes(), ccConfig.clientNetPort);
+    }
+
     private class DeadNodeSweeper extends TimerTask {
         @Override
         public void run() {
@@ -271,6 +297,10 @@
         }
     }
 
+    public IDatasetDirectoryService getDatasetDirectoryService() {
+        return datasetDirectoryService;
+    }
+
     private class HyracksClientInterfaceIPCI implements IIPCI {
         @Override
         public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception) {
@@ -285,27 +315,6 @@
                     return;
                 }
 
-                case CREATE_APPLICATION: {
-                    HyracksClientInterfaceFunctions.CreateApplicationFunction caf = (HyracksClientInterfaceFunctions.CreateApplicationFunction) fn;
-                    workQueue.schedule(new ApplicationCreateWork(ClusterControllerService.this, caf.getAppName(),
-                            new IPCResponder<Object>(handle, mid)));
-                    return;
-                }
-
-                case START_APPLICATION: {
-                    HyracksClientInterfaceFunctions.StartApplicationFunction saf = (HyracksClientInterfaceFunctions.StartApplicationFunction) fn;
-                    workQueue.schedule(new ApplicationStartWork(ClusterControllerService.this, saf.getAppName(),
-                            new IPCResponder<Object>(handle, mid)));
-                    return;
-                }
-
-                case DESTROY_APPLICATION: {
-                    HyracksClientInterfaceFunctions.DestroyApplicationFunction daf = (HyracksClientInterfaceFunctions.DestroyApplicationFunction) fn;
-                    workQueue.schedule(new ApplicationDestroyWork(ClusterControllerService.this, daf.getAppName(),
-                            new IPCResponder<Object>(handle, mid)));
-                    return;
-                }
-
                 case GET_JOB_STATUS: {
                     HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf = (HyracksClientInterfaceFunctions.GetJobStatusFunction) fn;
                     workQueue.schedule(new GetJobStatusWork(ClusterControllerService.this, gjsf.getJobId(),
@@ -316,8 +325,29 @@
                 case START_JOB: {
                     HyracksClientInterfaceFunctions.StartJobFunction sjf = (HyracksClientInterfaceFunctions.StartJobFunction) fn;
                     JobId jobId = createJobId();
-                    workQueue.schedule(new JobStartWork(ClusterControllerService.this, sjf.getAppName(), sjf
-                            .getACGGFBytes(), sjf.getJobFlags(), jobId, new IPCResponder<JobId>(handle, mid)));
+                    workQueue.schedule(new JobStartWork(ClusterControllerService.this, sjf.getACGGFBytes(), sjf
+                            .getJobFlags(), jobId, new IPCResponder<JobId>(handle, mid)));
+                    return;
+                }
+
+                case GET_DATASET_DIRECTORY_SERIVICE_INFO: {
+                    workQueue.schedule(new GetDatasetDirectoryServiceInfoWork(ClusterControllerService.this,
+                            new IPCResponder<NetworkAddress>(handle, mid)));
+                    return;
+                }
+
+                case GET_DATASET_RESULT_STATUS: {
+                    HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction gdrlf = (HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction) fn;
+                    workQueue.schedule(new GetResultStatusWork(ClusterControllerService.this, gdrlf.getJobId(), gdrlf
+                            .getResultSetId(), new IPCResponder<Status>(handle, mid)));
+                    return;
+                }
+
+                case GET_DATASET_RESULT_LOCATIONS: {
+                    HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction gdrlf = (HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction) fn;
+                    workQueue.schedule(new GetResultPartitionLocationsWork(ClusterControllerService.this, gdrlf
+                            .getJobId(), gdrlf.getResultSetId(), gdrlf.getKnownRecords(),
+                            new IPCResponder<DatasetDirectoryRecord[]>(handle, mid)));
                     return;
                 }
 
@@ -416,15 +446,32 @@
                     return;
                 }
 
-                case APPLICATION_STATE_CHANGE_RESPONSE: {
-                    CCNCFunctions.ApplicationStateChangeResponseFunction astrf = (CCNCFunctions.ApplicationStateChangeResponseFunction) fn;
-                    workQueue.schedule(new ApplicationStateChangeWork(ClusterControllerService.this, astrf));
+                case REGISTER_RESULT_PARTITION_LOCATION: {
+                    CCNCFunctions.RegisterResultPartitionLocationFunction rrplf = (CCNCFunctions.RegisterResultPartitionLocationFunction) fn;
+                    workQueue.schedule(new RegisterResultPartitionLocationWork(ClusterControllerService.this, rrplf
+                            .getJobId(), rrplf.getResultSetId(), rrplf.getOrderedResult(), rrplf.getPartition(), rrplf
+                            .getNPartitions(), rrplf.getNetworkAddress()));
                     return;
                 }
+
+                case REPORT_RESULT_PARTITION_WRITE_COMPLETION: {
+                    CCNCFunctions.ReportResultPartitionWriteCompletionFunction rrplf = (CCNCFunctions.ReportResultPartitionWriteCompletionFunction) fn;
+                    workQueue.schedule(new ReportResultPartitionWriteCompletionWork(ClusterControllerService.this,
+                            rrplf.getJobId(), rrplf.getResultSetId(), rrplf.getPartition()));
+                    return;
+                }
+
+                case REPORT_RESULT_PARTITION_FAILURE: {
+                    CCNCFunctions.ReportResultPartitionFailureFunction rrplf = (CCNCFunctions.ReportResultPartitionFailureFunction) fn;
+                    workQueue.schedule(new ReportResultPartitionFailureWork(ClusterControllerService.this, rrplf
+                            .getJobId(), rrplf.getResultSetId(), rrplf.getPartition()));
+                    return;
+                }
+
                 case SEND_APPLICATION_MESSAGE: {
                     CCNCFunctions.SendApplicationMessageFunction rsf = (CCNCFunctions.SendApplicationMessageFunction) fn;
                     workQueue.schedule(new ApplicationMessageWork(ClusterControllerService.this, rsf.getMessage(), rsf
-                            .getAppName(), rsf.getNodeId()));
+                            .getNodeId()));
                     return;
                 }
 
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
index c17acd0..c96a319 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
@@ -41,6 +41,8 @@
 
     private final NetworkAddress dataPort;
 
+    private final NetworkAddress datasetPort;
+
     private final Set<JobId> activeJobIds;
 
     private final String osName;
@@ -107,6 +109,14 @@
 
     private final long[] netSignalingBytesWritten;
 
+    private final long[] datasetNetPayloadBytesRead;
+
+    private final long[] datasetNetPayloadBytesWritten;
+
+    private final long[] datasetNetSignalingBytesRead;
+
+    private final long[] datasetNetSignalingBytesWritten;
+
     private final long[] ipcMessagesSent;
 
     private final long[] ipcMessageBytesSent;
@@ -123,6 +133,7 @@
         this.nodeController = nodeController;
         ncConfig = reg.getNCConfig();
         dataPort = reg.getDataPort();
+        datasetPort = reg.getDatasetPort();
         activeJobIds = new HashSet<JobId>();
 
         osName = reg.getOSName();
@@ -164,6 +175,10 @@
         netPayloadBytesWritten = new long[RRD_SIZE];
         netSignalingBytesRead = new long[RRD_SIZE];
         netSignalingBytesWritten = new long[RRD_SIZE];
+        datasetNetPayloadBytesRead = new long[RRD_SIZE];
+        datasetNetPayloadBytesWritten = new long[RRD_SIZE];
+        datasetNetSignalingBytesRead = new long[RRD_SIZE];
+        datasetNetSignalingBytesWritten = new long[RRD_SIZE];
         ipcMessagesSent = new long[RRD_SIZE];
         ipcMessageBytesSent = new long[RRD_SIZE];
         ipcMessagesReceived = new long[RRD_SIZE];
@@ -196,6 +211,10 @@
         netPayloadBytesWritten[rrdPtr] = hbData.netPayloadBytesWritten;
         netSignalingBytesRead[rrdPtr] = hbData.netSignalingBytesRead;
         netSignalingBytesWritten[rrdPtr] = hbData.netSignalingBytesWritten;
+        datasetNetPayloadBytesRead[rrdPtr] = hbData.datasetNetPayloadBytesRead;
+        datasetNetPayloadBytesWritten[rrdPtr] = hbData.datasetNetPayloadBytesWritten;
+        datasetNetSignalingBytesRead[rrdPtr] = hbData.datasetNetSignalingBytesRead;
+        datasetNetSignalingBytesWritten[rrdPtr] = hbData.datasetNetSignalingBytesWritten;
         ipcMessagesSent[rrdPtr] = hbData.ipcMessagesSent;
         ipcMessageBytesSent[rrdPtr] = hbData.ipcMessageBytesSent;
         ipcMessagesReceived[rrdPtr] = hbData.ipcMessagesReceived;
@@ -227,6 +246,10 @@
         return dataPort;
     }
 
+    public NetworkAddress getDatasetPort() {
+        return datasetPort;
+    }
+
     public JSONObject toSummaryJSON() throws JSONException {
         JSONObject o = new JSONObject();
         o.put("node-id", ncConfig.nodeId);
@@ -271,6 +294,10 @@
         o.put("net-payload-bytes-written", netPayloadBytesWritten);
         o.put("net-signaling-bytes-read", netSignalingBytesRead);
         o.put("net-signaling-bytes-written", netSignalingBytesWritten);
+        o.put("dataset-net-payload-bytes-read", datasetNetPayloadBytesRead);
+        o.put("dataset-net-payload-bytes-written", datasetNetPayloadBytesWritten);
+        o.put("dataset-net-signaling-bytes-read", datasetNetSignalingBytesRead);
+        o.put("dataset-net-signaling-bytes-written", datasetNetSignalingBytesWritten);
         o.put("ipc-messages-sent", ipcMessagesSent);
         o.put("ipc-message-bytes-sent", ipcMessageBytesSent);
         o.put("ipc-messages-received", ipcMessagesReceived);
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
index 24dfa7c..7e1581a 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
@@ -22,13 +22,11 @@
 import java.util.Set;
 
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.application.ICCBootstrap;
 import edu.uci.ics.hyracks.api.context.ICCContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
 import edu.uci.ics.hyracks.api.job.IJobLifecycleListener;
 import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
 import edu.uci.ics.hyracks.control.common.work.IResultCallback;
@@ -44,42 +42,18 @@
 
     private List<IJobLifecycleListener> jobLifecycleListeners;
 
-    public CCApplicationContext(ServerContext serverCtx, ICCContext ccContext, String appName) throws IOException {
-        super(serverCtx, appName);
+    public CCApplicationContext(ServerContext serverCtx, ICCContext ccContext) throws IOException {
+        super(serverCtx);
         this.ccContext = ccContext;
         initPendingNodeIds = new HashSet<String>();
         deinitPendingNodeIds = new HashSet<String>();
         jobLifecycleListeners = new ArrayList<IJobLifecycleListener>();
     }
 
-    @Override
-    protected void start() throws Exception {
-        ((ICCBootstrap) bootstrap).setApplicationContext(this);
-        bootstrap.start();
-    }
-
     public ICCContext getCCContext() {
         return ccContext;
     }
 
-    public IActivityClusterGraphGeneratorFactory createActivityClusterGraphGeneratorFactory(byte[] bytes)
-            throws HyracksException {
-        try {
-            return (IActivityClusterGraphGeneratorFactory) JavaSerializationUtils.deserialize(bytes, getClassLoader());
-        } catch (IOException e) {
-            throw new HyracksException(e);
-        } catch (ClassNotFoundException e) {
-            throw new HyracksException(e);
-        }
-    }
-
-    @Override
-    protected void stop() throws Exception {
-        if (bootstrap != null) {
-            bootstrap.stop();
-        }
-    }
-
     @Override
     public void setDistributedState(Serializable state) {
         this.distributedState = state;
@@ -108,28 +82,4 @@
             l.notifyJobCreation(jobId, acggf);
         }
     }
-
-    public Set<String> getInitializationPendingNodeIds() {
-        return initPendingNodeIds;
-    }
-
-    public Set<String> getDeinitializationPendingNodeIds() {
-        return deinitPendingNodeIds;
-    }
-
-    public IResultCallback<Object> getInitializationCallback() {
-        return initializationCallback;
-    }
-
-    public void setInitializationCallback(IResultCallback<Object> initializationCallback) {
-        this.initializationCallback = initializationCallback;
-    }
-
-    public IResultCallback<Object> getDeinitializationCallback() {
-        return deinitializationCallback;
-    }
-
-    public void setDeinitializationCallback(IResultCallback<Object> deinitializationCallback) {
-        this.deinitializationCallback = deinitializationCallback;
-    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java
new file mode 100644
index 0000000..13d0c30
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java
@@ -0,0 +1,241 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.dataset;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+/**
+ * TODO(madhusudancs): The potential perils of this global dataset directory service implementation is that, the jobs
+ * location information is never evicted from the memory and the memory usage grows as the number of jobs in the system
+ * grows. What we should possibly do is, add an API call for the client to say that it received everything it has to for
+ * the job (after it receives all the results) completely. Then we can just get rid of the location information for that
+ * job.
+ */
+public class DatasetDirectoryService implements IDatasetDirectoryService {
+    private final Map<JobId, Map<ResultSetId, ResultSetMetaData>> jobResultLocationsMap;
+
+    public DatasetDirectoryService() {
+        jobResultLocationsMap = new HashMap<JobId, Map<ResultSetId, ResultSetMetaData>>();
+    }
+
+    @Override
+    public synchronized void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult,
+            int partition, int nPartitions, NetworkAddress networkAddress) {
+        Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
+        if (rsMap == null) {
+            rsMap = new HashMap<ResultSetId, ResultSetMetaData>();
+            jobResultLocationsMap.put(jobId, rsMap);
+        }
+
+        ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+        if (resultSetMetaData == null) {
+            resultSetMetaData = new ResultSetMetaData(orderedResult, new DatasetDirectoryRecord[nPartitions]);
+            rsMap.put(rsId, resultSetMetaData);
+        }
+
+        DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+        if (records[partition] == null) {
+            records[partition] = new DatasetDirectoryRecord();
+        }
+        records[partition].setNetworkAddress(networkAddress);
+        records[partition].start();
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) {
+        DatasetDirectoryRecord ddr = getDatasetDirectoryRecord(jobId, rsId, partition);
+        ddr.writeEOS();
+    }
+
+    @Override
+    public synchronized void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) {
+        DatasetDirectoryRecord ddr = getDatasetDirectoryRecord(jobId, rsId, partition);
+        ddr.fail();
+    }
+
+    @Override
+    public synchronized Status getResultStatus(JobId jobId, ResultSetId rsId) throws HyracksDataException {
+        Map<ResultSetId, ResultSetMetaData> rsMap;
+        while ((rsMap = jobResultLocationsMap.get(jobId)) == null) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+        if (resultSetMetaData == null || resultSetMetaData.getRecords() == null) {
+            throw new HyracksDataException("ResultSet locations uninitialized when it is expected to be initialized.");
+        }
+        DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+
+        ArrayList<Status> statuses = new ArrayList<Status>(records.length);
+        for (int i = 0; i < records.length; i++) {
+            statuses.add(records[i].getStatus());
+        }
+
+        // Default status is idle
+        Status status = Status.IDLE;
+        if (statuses.contains(Status.FAILED)) {
+            // Even if there is at least one failed entry we should return failed status.
+            return Status.FAILED;
+        } else if (statuses.contains(Status.RUNNING)) {
+            // If there are not failed entry and if there is at least one running entry we should return running status.
+            return Status.RUNNING;
+        } else {
+            // If each and every partition has reported success do we report success as the status.
+            int successCount = 0;
+            for (int i = 0; i < statuses.size(); i++) {
+                if (statuses.get(i) == Status.SUCCESS) {
+                    successCount++;
+                }
+            }
+            if (successCount == statuses.size()) {
+                return Status.SUCCESS;
+            }
+        }
+        return status;
+    }
+
+    @Override
+    public synchronized DatasetDirectoryRecord[] getResultPartitionLocations(JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownRecords) throws HyracksDataException {
+        DatasetDirectoryRecord[] newRecords;
+        while ((newRecords = updatedRecords(jobId, rsId, knownRecords)) == null) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        return newRecords;
+    }
+
+    public DatasetDirectoryRecord getDatasetDirectoryRecord(JobId jobId, ResultSetId rsId, int partition) {
+        Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
+        ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+        DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+        return records[partition];
+    }
+
+    /**
+     * Compares the records already known by the client for the given job's result set id with the records that the
+     * dataset directory service knows and if there are any newly discovered records returns a whole array with the
+     * new records filled in.
+     * This method has a very convoluted logic. Here is the explanation of how it works.
+     * If the ordering constraint has to be enforced, the method obtains the first null record in the known records in
+     * the order of the partitions. It always traverses the array in the first to last order!
+     * If known records array or the first element in that array is null in the but the record for that partition now
+     * known to the directory service, the method fills in that record in the array and returns the array back.
+     * However, if the first known null record is not a first element in the array, by induction, all the previous
+     * known records should be known already be known to client and none of the records for the partitions ahead is
+     * known by the client yet. So, we check if the client has reached the end of stream for the partition corresponding
+     * to the record before the first known null record, i.e. the last known non-null record. If not, we just return
+     * null because we cannot expose any new locations until the client reaches end of stream for the last known record.
+     * If the client has reached the end of stream record for the last known non-null record, we check if the next record
+     * is discovered by the dataset directory service and if so, we fill it in the records array and return it back or
+     * send null otherwise.
+     * If the ordering is not required, we are free to return any newly discovered records back, so we just check if
+     * arrays are equal and if they are not we send the entire new updated array.
+     * 
+     * @param jobId
+     *            - Id of the job for which the directory records should be retrieved.
+     * @param rsId
+     *            - Id of the result set for which the directory records should be retrieved.
+     * @param knownRecords
+     *            - An array of directory records that the client is already aware of.
+     * @return
+     *         - Returns null if there aren't any newly discovered partitions enforcing the ordering constraint
+     * @throws HyracksDataException
+     *             TODO(madhusudancs): Think about caching (and still be stateless) instead of this ugly O(n) iterations for
+     *             every check. This already looks very expensive.
+     */
+    private DatasetDirectoryRecord[] updatedRecords(JobId jobId, ResultSetId rsId, DatasetDirectoryRecord[] knownRecords)
+            throws HyracksDataException {
+        Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
+        if (rsMap == null) {
+            return null;
+        }
+
+        ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+        if (resultSetMetaData == null || resultSetMetaData.getRecords() == null) {
+            throw new HyracksDataException("ResultSet locations uninitialized when it is expected to be initialized.");
+        }
+
+        boolean ordered = resultSetMetaData.getOrderedResult();
+        DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+        /* If ordering is required, we should expose the dataset directory records only in the order, otherwise
+         * we can simply check if there are any newly discovered records and send the whole array back if there are.
+         */
+        if (ordered) {
+            // Iterate over the known records and find the last record which is not null.
+            int i = 0;
+            for (i = 0; i < records.length; i++) {
+                if (knownRecords == null) {
+                    if (records[0] != null) {
+                        knownRecords = new DatasetDirectoryRecord[records.length];
+                        knownRecords[0] = records[0];
+                        return knownRecords;
+                    }
+                    return null;
+                }
+                if (knownRecords[i] == null) {
+                    if ((i == 0 || knownRecords[i - 1].hasReachedReadEOS()) && records[i] != null) {
+                        knownRecords[i] = records[i];
+                        return knownRecords;
+                    }
+                    return null;
+                }
+            }
+        } else {
+            if (!Arrays.equals(records, knownRecords)) {
+                return records;
+            }
+        }
+        return null;
+    }
+
+    private class ResultSetMetaData {
+        private final boolean ordered;
+
+        private final DatasetDirectoryRecord[] records;
+
+        public ResultSetMetaData(boolean ordered, DatasetDirectoryRecord[] records) {
+            this.ordered = ordered;
+            this.records = records;
+        }
+
+        public boolean getOrderedResult() {
+            return ordered;
+        }
+
+        public DatasetDirectoryRecord[] getRecords() {
+            return records;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
index 6f26de2..33e1ff6 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
@@ -47,8 +47,6 @@
 public class JobRun implements IJobStatusConditionVariable {
     private final JobId jobId;
 
-    private final String applicationName;
-
     private final IActivityClusterGraphGenerator acgg;
 
     private final ActivityClusterGraph acg;
@@ -83,10 +81,9 @@
 
     private Exception pendingException;
 
-    public JobRun(ClusterControllerService ccs, JobId jobId, String applicationName,
-            IActivityClusterGraphGenerator acgg, EnumSet<JobFlag> jobFlags) {
+    public JobRun(ClusterControllerService ccs, JobId jobId, IActivityClusterGraphGenerator acgg,
+            EnumSet<JobFlag> jobFlags) {
         this.jobId = jobId;
-        this.applicationName = applicationName;
         this.acgg = acgg;
         this.acg = acgg.initialize();
         this.scheduler = new JobScheduler(ccs, this, acgg.getConstraints());
@@ -103,10 +100,6 @@
         return jobId;
     }
 
-    public String getApplicationName() {
-        return applicationName;
-    }
-
     public ActivityClusterGraph getActivityClusterGraph() {
         return acg;
     }
@@ -208,7 +201,6 @@
         JSONObject result = new JSONObject();
 
         result.put("job-id", jobId.toString());
-        result.put("application-name", applicationName);
         result.put("status", getStatus());
         result.put("create-time", getCreateTime());
         result.put("start-time", getCreateTime());
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
index b163db5..34b7dc7 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
@@ -429,7 +429,6 @@
     private void startTasks(Map<String, List<TaskAttemptDescriptor>> taskAttemptMap) throws HyracksException {
         final JobId jobId = jobRun.getJobId();
         final ActivityClusterGraph acg = jobRun.getActivityClusterGraph();
-        final String appName = jobRun.getApplicationName();
         final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = new HashMap<ConnectorDescriptorId, IConnectorPolicy>(
                 jobRun.getConnectorPolicyMap());
         for (Map.Entry<String, List<TaskAttemptDescriptor>> entry : taskAttemptMap.entrySet()) {
@@ -444,7 +443,7 @@
                 }
                 try {
                     byte[] jagBytes = changed ? JavaSerializationUtils.serialize(acg) : null;
-                    node.getNodeController().startTasks(appName, jobId, jagBytes, taskDescriptors, connectorPolicies,
+                    node.getNodeController().startTasks(jobId, jagBytes, taskDescriptors, connectorPolicies,
                             jobRun.getFlags());
                 } catch (Exception e) {
                     e.printStackTrace();
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
deleted file mode 100644
index 73f3a1a0..0000000
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
+++ /dev/null
@@ -1,117 +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.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.common.application.ApplicationContext;
-import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
-
-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 SynchronizableWork {
-                    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.getWorkQueue().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 SynchronizableWork {
-                    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.getWorkQueue().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/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
index c32cb97..e39e766 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
@@ -70,10 +70,6 @@
 
         addHandler(createAdminConsoleHandler());
         addHandler(createStaticResourcesHandler());
-
-        handler = new ContextHandler("/applications");
-        handler.setHandler(new ApplicationInstallationHandler(ccs));
-        addHandler(handler);
     }
 
     private Handler createAdminConsoleHandler() {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java
deleted file mode 100644
index 15d6d1f..0000000
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java
+++ /dev/null
@@ -1,61 +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.work;
-
-import java.io.IOException;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.IResultCallback;
-
-public class ApplicationCreateWork extends AbstractWork {
-    private final ClusterControllerService ccs;
-    private final String appName;
-    private IResultCallback<Object> callback;
-
-    public ApplicationCreateWork(ClusterControllerService ccs, String appName, IResultCallback<Object> callback) {
-        this.ccs = ccs;
-        this.appName = appName;
-        this.callback = callback;
-    }
-
-    @Override
-    public void run() {
-        try {
-            Map<String, CCApplicationContext> applications = ccs.getApplicationMap();
-            if (applications.containsKey(appName)) {
-                callback.setException(new HyracksException("Duplicate application with name: " + appName
-                        + " being created."));
-                return;
-            }
-            CCApplicationContext appCtx;
-            try {
-                appCtx = new CCApplicationContext(ccs.getServerContext(), ccs.getCCContext(), appName);
-            } catch (IOException e) {
-                callback.setException(e);
-                return;
-            }
-            appCtx.setStatus(ApplicationStatus.CREATED);
-            applications.put(appName, appCtx);
-            callback.setValue(null);
-        } catch (Exception e) {
-            callback.setException(e);
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
deleted file mode 100644
index fc86c4c..0000000
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
+++ /dev/null
@@ -1,64 +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.work;
-
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-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.common.application.ApplicationStatus;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.IResultCallback;
-
-public class ApplicationDestroyWork extends AbstractWork {
-    private final ClusterControllerService ccs;
-    private final String appName;
-    private IResultCallback<Object> callback;
-
-    public ApplicationDestroyWork(ClusterControllerService ccs, String appName, IResultCallback<Object> callback) {
-        this.ccs = ccs;
-        this.appName = appName;
-        this.callback = callback;
-    }
-
-    @Override
-    public void run() {
-        try {
-            final CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
-            if (appCtx == null) {
-                callback.setException(new HyracksException("No application with name: " + appName));
-                return;
-            }
-            if (appCtx.getStatus() == ApplicationStatus.IN_DEINITIALIZATION
-                    || appCtx.getStatus() == ApplicationStatus.DEINITIALIZED) {
-                return;
-            }
-            Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
-            appCtx.getDeinitializationPendingNodeIds().addAll(nodeMap.keySet());
-            appCtx.setStatus(ApplicationStatus.IN_DEINITIALIZATION);
-            appCtx.setDeinitializationCallback(callback);
-            for (String nodeId : ccs.getNodeMap().keySet()) {
-                NodeControllerState nodeState = nodeMap.get(nodeId);
-                final INodeController node = nodeState.getNodeController();
-                node.destroyApplication(appName);
-            }
-        } catch (Exception e) {
-            callback.setException(e);
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java
index 22ff84d..b880c8a 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java
@@ -18,14 +18,14 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
 import edu.uci.ics.hyracks.api.messages.IMessage;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
 /**
  * @author rico
- * 
  */
 public class ApplicationMessageWork extends AbstractWork {
 
@@ -33,26 +33,24 @@
     private byte[] message;
     private String nodeId;
     private ClusterControllerService ccs;
-    private String appName;
 
-    public ApplicationMessageWork(ClusterControllerService ccs, byte[] message, String appName, String nodeId) {
+    public ApplicationMessageWork(ClusterControllerService ccs, byte[] message, String nodeId) {
         this.ccs = ccs;
         this.nodeId = nodeId;
         this.message = message;
-        this.appName = appName;
     }
 
     @Override
     public void run() {
-
-        final ApplicationContext ctx = ccs.getApplicationMap().get(appName);
+        final ICCApplicationContext ctx = ccs.getApplicationContext();
         try {
-            final IMessage data = (IMessage) ctx.deserialize(message);
-            (new Thread() {
+            final IMessage data = (IMessage) JavaSerializationUtils.deserialize(message);
+            ccs.getExecutor().execute(new Runnable() {
+                @Override
                 public void run() {
                     ctx.getMessageBroker().receivedMessage(data, nodeId);
                 }
-            }).start();
+            });
         } catch (IOException e) {
             LOGGER.log(Level.WARNING, "Error in stats reporting", e);
         } catch (ClassNotFoundException e) {
@@ -64,5 +62,4 @@
     public String toString() {
         return "nodeID: " + nodeId;
     }
-
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
deleted file mode 100644
index e4ad56c..0000000
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
+++ /dev/null
@@ -1,80 +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.work;
-
-import java.util.HashMap;
-import java.util.Map;
-
-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.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.IResultCallback;
-
-public class ApplicationStartWork extends AbstractWork {
-    private final ClusterControllerService ccs;
-    private final String appName;
-    private final IResultCallback<Object> callback;
-
-    public ApplicationStartWork(ClusterControllerService ccs, String appName, IResultCallback<Object> callback) {
-        this.ccs = ccs;
-        this.appName = appName;
-        this.callback = callback;
-    }
-
-    @Override
-    public void run() {
-        try {
-            final CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
-            if (appCtx == null) {
-                callback.setException(new HyracksException("No application with name: " + appName));
-                return;
-            }
-            if (appCtx.getStatus() != ApplicationStatus.CREATED) {
-                callback.setException(new HyracksException("Application in incorrect state for starting: "
-                        + appCtx.getStatus()));
-            }
-            final Map<String, NodeControllerState> nodeMapCopy = new HashMap<String, NodeControllerState>(
-                    ccs.getNodeMap());
-            appCtx.getInitializationPendingNodeIds().addAll(nodeMapCopy.keySet());
-            appCtx.setStatus(ApplicationStatus.IN_INITIALIZATION);
-            appCtx.setInitializationCallback(callback);
-            ccs.getExecutor().execute(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        appCtx.initializeClassPath();
-                        appCtx.initialize();
-                        final byte[] distributedState = JavaSerializationUtils.serialize(appCtx.getDistributedState());
-                        final boolean deployHar = appCtx.containsHar();
-                        for (final String nodeId : nodeMapCopy.keySet()) {
-                            NodeControllerState nodeState = nodeMapCopy.get(nodeId);
-                            final INodeController node = nodeState.getNodeController();
-                            node.createApplication(appName, deployHar, distributedState);
-                        }
-                    } catch (Exception e) {
-                        callback.setException(e);
-                    }
-                }
-            });
-        } catch (Exception e) {
-            callback.setException(e);
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStateChangeWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStateChangeWork.java
deleted file mode 100644
index f6271fe..0000000
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStateChangeWork.java
+++ /dev/null
@@ -1,85 +0,0 @@
-package edu.uci.ics.hyracks.control.cc.work;
-
-import java.util.Set;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
-import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.IResultCallback;
-
-public class ApplicationStateChangeWork extends AbstractWork {
-    private static final Logger LOGGER = Logger.getLogger(ApplicationStateChangeWork.class.getName());
-
-    private final ClusterControllerService ccs;
-    private final CCNCFunctions.ApplicationStateChangeResponseFunction ascrf;
-
-    public ApplicationStateChangeWork(ClusterControllerService ccs,
-            CCNCFunctions.ApplicationStateChangeResponseFunction ascrf) {
-        this.ccs = ccs;
-        this.ascrf = ascrf;
-    }
-
-    @Override
-    public void run() {
-        final CCApplicationContext appCtx = ccs.getApplicationMap().get(ascrf.getApplicationName());
-        if (appCtx == null) {
-            LOGGER.warning("Got ApplicationStateChangeResponse for application " + ascrf.getApplicationName()
-                    + " that does not exist");
-            return;
-        }
-        switch (ascrf.getStatus()) {
-            case INITIALIZED: {
-                Set<String> pendingNodeIds = appCtx.getInitializationPendingNodeIds();
-                boolean changed = pendingNodeIds.remove(ascrf.getNodeId());
-                if (!changed) {
-                    LOGGER.warning("Got ApplicationStateChangeResponse for application " + ascrf.getApplicationName()
-                            + " from unexpected node " + ascrf.getNodeId() + " to state " + ascrf.getStatus());
-                    return;
-                }
-                if (pendingNodeIds.isEmpty()) {
-                    appCtx.setStatus(ApplicationStatus.INITIALIZED);
-                    IResultCallback<Object> callback = appCtx.getInitializationCallback();
-                    appCtx.setInitializationCallback(null);
-                    callback.setValue(null);
-                }
-                return;
-            }
-
-            case DEINITIALIZED: {
-                Set<String> pendingNodeIds = appCtx.getDeinitializationPendingNodeIds();
-                boolean changed = pendingNodeIds.remove(ascrf.getNodeId());
-                if (!changed) {
-                    LOGGER.warning("Got ApplicationStateChangeResponse for application " + ascrf.getApplicationName()
-                            + " from unexpected node " + ascrf.getNodeId() + " to state " + ascrf.getStatus());
-                    return;
-                }
-                if (pendingNodeIds.isEmpty()) {
-                    appCtx.setStatus(ApplicationStatus.DEINITIALIZED);
-                    ccs.getExecutor().execute(new Runnable() {
-                        @Override
-                        public void run() {
-                            try {
-                                appCtx.deinitialize();
-                            } catch (Exception e) {
-                                e.printStackTrace();
-                            }
-                            ccs.getWorkQueue().schedule(new AbstractWork() {
-                                @Override
-                                public void run() {
-                                    ccs.getApplicationMap().remove(ascrf.getApplicationName());
-                                    IResultCallback<Object> callback = appCtx.getDeinitializationCallback();
-                                    appCtx.setDeinitializationCallback(null);
-                                    callback.setValue(null);
-                                }
-                            });
-                        }
-                    });
-                }
-                return;
-            }
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java
new file mode 100644
index 0000000..3ac6acc
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class GetDatasetDirectoryServiceInfoWork extends SynchronizableWork {
+    private final ClusterControllerService ccs;
+
+    private final IResultCallback<NetworkAddress> callback;
+
+    public GetDatasetDirectoryServiceInfoWork(ClusterControllerService ccs, IResultCallback<NetworkAddress> callback) {
+        this.ccs = ccs;
+        this.callback = callback;
+    }
+
+    @Override
+    public void doRun() {
+        try {
+            NetworkAddress addr = ccs.getDatasetDirectoryServiceInfo();
+            callback.setValue(addr);
+        } catch (Exception e) {
+            callback.setException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSummariesJSONWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSummariesJSONWork.java
index a0afd61..20035a4 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSummariesJSONWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSummariesJSONWork.java
@@ -44,7 +44,6 @@
             JSONObject jo = new JSONObject();
             jo.put("type", "job-summary");
             jo.put("job-id", run.getJobId().toString());
-            jo.put("application-name", run.getApplicationName());
             jo.put("create-time", run.getCreateTime());
             jo.put("start-time", run.getCreateTime());
             jo.put("end-time", run.getCreateTime());
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
index 2f23a2c..a787b9f 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
@@ -39,7 +39,8 @@
         Map<String, NodeControllerInfo> result = new LinkedHashMap<String, NodeControllerInfo>();
         Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
         for (Map.Entry<String, NodeControllerState> e : nodeMap.entrySet()) {
-            result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE, e.getValue().getDataPort()));
+            result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE, e.getValue().getDataPort(), e
+                    .getValue().getDatasetPort()));
         }
         callback.setValue(result);
     }
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultPartitionLocationsWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
new file mode 100644
index 0000000..fd1d418
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class GetResultPartitionLocationsWork extends SynchronizableWork {
+    private final ClusterControllerService ccs;
+
+    private final JobId jobId;
+
+    private final ResultSetId rsId;
+
+    private final DatasetDirectoryRecord[] knownRecords;
+
+    private final IResultCallback<DatasetDirectoryRecord[]> callback;
+
+    public GetResultPartitionLocationsWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+            DatasetDirectoryRecord[] knownRecords, IResultCallback<DatasetDirectoryRecord[]> callback) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.rsId = rsId;
+        this.knownRecords = knownRecords;
+        this.callback = callback;
+    }
+
+    @Override
+    public void doRun() {
+        final IDatasetDirectoryService dds = ccs.getDatasetDirectoryService();
+        ccs.getExecutor().execute(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    DatasetDirectoryRecord[] partitionLocations = dds.getResultPartitionLocations(jobId, rsId,
+                            knownRecords);
+                    callback.setValue(partitionLocations);
+                } catch (HyracksDataException e) {
+                    callback.setException(e);
+                }
+            }
+        });
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java
new file mode 100644
index 0000000..d2dadf5
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class GetResultStatusWork extends SynchronizableWork {
+    private final ClusterControllerService ccs;
+
+    private final JobId jobId;
+
+    private final ResultSetId rsId;
+
+    private final IResultCallback<Status> callback;
+
+    public GetResultStatusWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+            IResultCallback<Status> callback) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.rsId = rsId;
+        this.callback = callback;
+    }
+
+    @Override
+    public void doRun() {
+        try {
+            Status status = ccs.getDatasetDirectoryService().getResultStatus(jobId, rsId);
+            callback.setValue(status);
+        } catch (HyracksDataException e) {
+            callback.setException(e);
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "JobId@" + jobId + " ResultSetId@" + rsId;
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
index 6b5ff03..b304b21 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
@@ -69,7 +69,7 @@
                 }
             }
         } else {
-            CCApplicationContext appCtx = ccs.getApplicationMap().get(run.getApplicationName());
+            CCApplicationContext appCtx = ccs.getApplicationContext();
             if (appCtx != null) {
                 try {
                     appCtx.notifyJobFinish(jobId);
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
index b6a33cd..f7d6c4c 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
@@ -16,12 +16,12 @@
 
 import java.util.EnumSet;
 
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGenerator;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
 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.JobRun;
@@ -33,30 +33,25 @@
     private final byte[] acggfBytes;
     private final EnumSet<JobFlag> jobFlags;
     private final JobId jobId;
-    private final String appName;
     private final IResultCallback<JobId> callback;
 
-    public JobStartWork(ClusterControllerService ccs, String appName, byte[] acggfBytes, EnumSet<JobFlag> jobFlags,
-            JobId jobId, IResultCallback<JobId> callback) {
+    public JobStartWork(ClusterControllerService ccs, byte[] acggfBytes, EnumSet<JobFlag> jobFlags, JobId jobId,
+            IResultCallback<JobId> callback) {
         this.jobId = jobId;
         this.ccs = ccs;
         this.acggfBytes = acggfBytes;
         this.jobFlags = jobFlags;
-        this.appName = appName;
         this.callback = callback;
     }
 
     @Override
     protected void doRun() throws Exception {
         try {
-            final CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
-            if (appCtx == null) {
-                throw new HyracksException("No application with id " + appName + " found");
-            }
-            IActivityClusterGraphGeneratorFactory acggf = appCtx.createActivityClusterGraphGeneratorFactory(acggfBytes);
-            IActivityClusterGraphGenerator acgg = acggf.createActivityClusterGraphGenerator(appName, jobId, appCtx,
-                    jobFlags);
-            JobRun run = new JobRun(ccs, jobId, appName, acgg, jobFlags);
+            final CCApplicationContext appCtx = ccs.getApplicationContext();
+            IActivityClusterGraphGeneratorFactory acggf = (IActivityClusterGraphGeneratorFactory) JavaSerializationUtils
+                    .deserialize(acggfBytes);
+            IActivityClusterGraphGenerator acgg = acggf.createActivityClusterGraphGenerator(jobId, appCtx, jobFlags);
+            JobRun run = new JobRun(ccs, jobId, acgg, jobFlags);
             run.setStatus(JobStatus.INITIALIZED, null);
             ccs.getActiveRunMap().put(jobId, run);
             appCtx.notifyJobCreation(jobId, acggf);
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
index 9927289..ed58c43 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
@@ -59,7 +59,7 @@
             ncs.getActiveJobIds().remove(jobId);
         }
         if (cleanupPendingNodes.isEmpty()) {
-            CCApplicationContext appCtx = ccs.getApplicationMap().get(run.getApplicationName());
+            CCApplicationContext appCtx = ccs.getApplicationContext();
             if (appCtx != null) {
                 try {
                     appCtx.notifyJobFinish(jobId);
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
index e4e0aae..f7dd1d2 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
@@ -67,6 +67,7 @@
             LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
             NodeParameters params = new NodeParameters();
             params.setClusterControllerInfo(ccs.getClusterControllerInfo());
+            params.setDistributedState(ccs.getApplicationContext().getDistributedState());
             params.setHeartbeatPeriod(ccs.getCCConfig().heartbeatPeriod);
             params.setProfileDumpPeriod(ccs.getCCConfig().profileDumpPeriod);
             result = new CCNCFunctions.NodeRegistrationResult(params, null);
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
new file mode 100644
index 0000000..f86e924
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class RegisterResultPartitionLocationWork extends AbstractWork {
+    private final ClusterControllerService ccs;
+
+    private final JobId jobId;
+
+    private final ResultSetId rsId;
+
+    private final boolean orderedResult;
+
+    private final int partition;
+
+    private final int nPartitions;
+
+    private final NetworkAddress networkAddress;
+
+    public RegisterResultPartitionLocationWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+            boolean orderedResult, int partition, int nPartitions, NetworkAddress networkAddress) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.rsId = rsId;
+        this.orderedResult = orderedResult;
+        this.partition = partition;
+        this.nPartitions = nPartitions;
+        this.networkAddress = networkAddress;
+    }
+
+    @Override
+    public void run() {
+        ccs.getDatasetDirectoryService().registerResultPartitionLocation(jobId, rsId, orderedResult, partition,
+                nPartitions, networkAddress);
+    }
+
+    @Override
+    public String toString() {
+        return "JobId@" + jobId + " ResultSetId@" + rsId + " Partition@" + partition + " NPartitions@" + nPartitions
+                + " ResultPartitionLocation@" + networkAddress + " OrderedResult@" + orderedResult;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionFailureWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionFailureWork.java
new file mode 100644
index 0000000..4aea41e
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionFailureWork.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class ReportResultPartitionFailureWork extends AbstractWork {
+    private final ClusterControllerService ccs;
+
+    private final JobId jobId;
+
+    private final ResultSetId rsId;
+
+    private final int partition;
+
+    public ReportResultPartitionFailureWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId, int partition) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.rsId = rsId;
+        this.partition = partition;
+    }
+
+    @Override
+    public void run() {
+        ccs.getDatasetDirectoryService().reportResultPartitionFailure(jobId, rsId, partition);
+    }
+
+    @Override
+    public String toString() {
+        return "JobId@" + jobId + " ResultSetId@" + rsId + " Partition@" + partition;
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java
new file mode 100644
index 0000000..313b730
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class ReportResultPartitionWriteCompletionWork extends AbstractWork {
+    private final ClusterControllerService ccs;
+
+    private final JobId jobId;
+
+    private final ResultSetId rsId;
+
+    private final int partition;
+
+    public ReportResultPartitionWriteCompletionWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+            int partition) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.rsId = rsId;
+        this.partition = partition;
+    }
+
+    @Override
+    public void run() {
+        ccs.getDatasetDirectoryService().reportResultPartitionWriteCompletion(jobId, rsId, partition);
+    }
+
+    @Override
+    public String toString() {
+        return "JobId@" + jobId + " ResultSetId@" + rsId + " Partition@" + partition;
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js b/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js
index ff9d8a0..3fc46ff 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js
@@ -58,6 +58,10 @@
         var netPayloadBytesWritten = result['net-payload-bytes-written'];
         var netSignalingBytesRead = result['net-signaling-bytes-read'];
         var netSignalingBytesWritten = result['net-signaling-bytes-written'];
+        var datasetNetPayloadBytesRead = result['dataset-net-payload-bytes-read'];
+        var datasetNetPayloadBytesWritten = result['dataset-net-payload-bytes-written'];
+        var datasetNetSignalingBytesRead = result['dataset-net-signaling-bytes-read'];
+        var datasetNetSignalingBytesWritten = result['dataset-net-signaling-bytes-written'];
         var ipcMessagesSent = result['ipc-messages-sent'];
         var ipcMessageBytesSent = result['ipc-message-bytes-sent'];
         var ipcMessagesReceived = result['ipc-messages-received'];
@@ -117,9 +121,13 @@
             }
             if (i < sysLoad.length - 1) {
                 netPayloadReadBWArray.push([ i, computeRate(netPayloadBytesRead, rrdPtr) ]);
+                netPayloadReadBWArray.push([ i, computeRate(datasetNetPayloadBytesRead, rrdPtr) ]);
                 netPayloadWriteBWArray.push([ i, computeRate(netPayloadBytesWritten, rrdPtr) ]);
+                netPayloadWriteBWArray.push([ i, computeRate(datasetNetPayloadBytesWritten, rrdPtr) ]);
                 netSignalingReadBWArray.push([ i, computeRate(netSignalingBytesRead, rrdPtr) ]);
+                netSignalingReadBWArray.push([ i, computeRate(datasetNetSignalingBytesRead, rrdPtr) ]);
                 netSignalingWriteBWArray.push([ i, computeRate(netSignalingBytesWritten, rrdPtr) ]);
+                netSignalingWriteBWArray.push([ i, computeRate(etSignalingBytesWritten, rrdPtr) ]);
                 ipcMessageSendRateArray.push([ i, computeRate(ipcMessagesSent, rrdPtr) ]);
                 ipcMessageBytesSendRateArray.push([ i, computeRate(ipcMessageBytesSent, rrdPtr) ]);
                 ipcMessageReceiveRateArray.push([ i, computeRate(ipcMessagesReceived, rrdPtr) ]);
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
index fc755a5..5251584 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
@@ -14,180 +14,20 @@
  */
 package edu.uci.ics.hyracks.control.common.application;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.io.Serializable;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Enumeration;
-import java.util.List;
-import java.util.Properties;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipFile;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.IOUtils;
 
 import edu.uci.ics.hyracks.api.application.IApplicationContext;
-import edu.uci.ics.hyracks.api.application.IBootstrap;
 import edu.uci.ics.hyracks.api.messages.IMessageBroker;
-import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
 
 public abstract class ApplicationContext implements IApplicationContext {
-    private static final String APPLICATION_ROOT = "applications";
-    private static final String CLUSTER_CONTROLLER_BOOTSTRAP_CLASS_KEY = "cc.bootstrap.class";
-    private static final String NODE_CONTROLLER_BOOTSTRAP_CLASS_KEY = "nc.bootstrap.class";
-
     protected ServerContext serverCtx;
-    protected final String appName;
-    protected final File applicationRootDir;
-    protected ClassLoader classLoader;
-    protected ApplicationStatus status;
-    protected Properties deploymentDescriptor;
-    protected IBootstrap bootstrap;
     protected Serializable distributedState;
     protected IMessageBroker messageBroker;
 
-    public ApplicationContext(ServerContext serverCtx, String appName) throws IOException {
+    public ApplicationContext(ServerContext serverCtx) throws IOException {
         this.serverCtx = serverCtx;
-        this.appName = appName;
-        this.applicationRootDir = new File(new File(serverCtx.getBaseDir(), APPLICATION_ROOT), appName);
-        FileUtils.deleteDirectory(applicationRootDir);
-        applicationRootDir.mkdirs();
-    }
-
-    public String getApplicationName() {
-        return appName;
-    }
-
-    public void initializeClassPath() throws Exception {
-        if (expandArchive()) {
-            File expandedFolder = getExpandedFolder();
-            List<URL> urls = new ArrayList<URL>();
-            findJarFiles(expandedFolder, urls);
-            Collections.sort(urls, new Comparator<URL>() {
-                @Override
-                public int compare(URL o1, URL o2) {
-                    return o1.getFile().compareTo(o2.getFile());
-                }
-            });
-            classLoader = new URLClassLoader(urls.toArray(new URL[urls.size()]));
-        } else {
-            classLoader = getClass().getClassLoader();
-        }
-        deploymentDescriptor = parseDeploymentDescriptor();
-    }
-
-    public void initialize() throws Exception {
-        if (deploymentDescriptor != null) {
-            String bootstrapClass = null;
-            switch (serverCtx.getServerType()) {
-                case CLUSTER_CONTROLLER: {
-                    bootstrapClass = deploymentDescriptor.getProperty(CLUSTER_CONTROLLER_BOOTSTRAP_CLASS_KEY);
-                    break;
-                }
-                case NODE_CONTROLLER: {
-                    bootstrapClass = deploymentDescriptor.getProperty(NODE_CONTROLLER_BOOTSTRAP_CLASS_KEY);
-                    break;
-                }
-            }
-            if (bootstrapClass != null) {
-                bootstrap = (IBootstrap) classLoader.loadClass(bootstrapClass).newInstance();
-                start();
-            }
-        }
-    }
-
-    protected abstract void start() throws Exception;
-
-    protected abstract void stop() throws Exception;
-
-    private void findJarFiles(File dir, List<URL> urls) throws MalformedURLException {
-        for (File f : dir.listFiles()) {
-            if (f.isDirectory()) {
-                findJarFiles(f, urls);
-            } else if (f.getName().endsWith(".jar") || f.getName().endsWith(".zip")) {
-                urls.add(f.toURI().toURL());
-            }
-        }
-    }
-
-    private Properties parseDeploymentDescriptor() throws IOException {
-        InputStream in = classLoader.getResourceAsStream("hyracks-deployment.properties");
-        Properties props = new Properties();
-        if (in != null) {
-            try {
-                props.load(in);
-            } finally {
-                in.close();
-            }
-        }
-        return props;
-    }
-
-    private boolean expandArchive() throws IOException {
-        File archiveFile = getArchiveFile();
-        if (archiveFile.exists()) {
-            File expandedFolder = getExpandedFolder();
-            FileUtils.deleteDirectory(expandedFolder);
-            ZipFile zf = new ZipFile(archiveFile);
-            for (Enumeration<? extends ZipEntry> i = zf.entries(); i.hasMoreElements();) {
-                ZipEntry ze = i.nextElement();
-                String name = ze.getName();
-                if (name.endsWith("/")) {
-                    continue;
-                }
-                InputStream is = zf.getInputStream(ze);
-                OutputStream os = FileUtils.openOutputStream(new File(expandedFolder, name));
-                try {
-                    IOUtils.copyLarge(is, os);
-                } finally {
-                    os.close();
-                    is.close();
-                }
-            }
-            return true;
-        }
-        return false;
-    }
-
-    private File getExpandedFolder() {
-        return new File(applicationRootDir, "expanded");
-    }
-
-    public void deinitialize() throws Exception {
-        stop();
-        File expandedFolder = getExpandedFolder();
-        FileUtils.deleteDirectory(expandedFolder);
-    }
-
-    public Object deserialize(byte[] bytes) throws IOException, ClassNotFoundException {
-        return JavaSerializationUtils.deserialize(bytes, classLoader);
-    }
-
-    public OutputStream getHarOutputStream() throws IOException {
-        return new FileOutputStream(getArchiveFile());
-    }
-
-    private File getArchiveFile() {
-        return new File(applicationRootDir, "application.har");
-    }
-
-    public InputStream getHarInputStream() throws IOException {
-        return new FileInputStream(getArchiveFile());
-    }
-
-    public boolean containsHar() {
-        return getArchiveFile().exists();
     }
 
     @Override
@@ -196,19 +36,6 @@
     }
 
     @Override
-    public ClassLoader getClassLoader() {
-        return classLoader;
-    }
-
-    public void setStatus(ApplicationStatus status) {
-        this.status = status;
-    }
-
-    public ApplicationStatus getStatus() {
-        return status;
-    }
-
-    @Override
     public void setMessageBroker(IMessageBroker messageBroker) {
         this.messageBroker = messageBroker;
     }
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
index 0c5bb2f..627dd55 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
@@ -16,9 +16,10 @@
 
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
 import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
 import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
@@ -46,9 +47,14 @@
 
     public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception;
 
-    public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception;
+    public void sendApplicationMessageToCC(byte[] data, String nodeId) throws Exception;
 
-    public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception;
+    public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
+            int nPartitions, NetworkAddress networkAddress) throws Exception;
+
+    public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws Exception;
+
+    public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws Exception;
 
     public void getNodeControllerInfos() throws Exception;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
index 049adf8..c589c97 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
@@ -29,17 +29,12 @@
 import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
 
 public interface INodeController {
-    public void startTasks(String appName, JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
+    public void startTasks(JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
             Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, EnumSet<JobFlag> flags) throws Exception;
 
     public void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception;
 
     public void cleanUpJoblet(JobId jobId, JobStatus status) throws Exception;
 
-    public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
-            throws Exception;
-
-    public void destroyApplication(String appName) throws Exception;
-
     public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
index 6c208fe..7647410 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
@@ -17,7 +17,9 @@
 import java.io.File;
 import java.util.List;
 
+import org.kohsuke.args4j.Argument;
 import org.kohsuke.args4j.Option;
+import org.kohsuke.args4j.spi.StopOptionHandler;
 
 public class CCConfig {
     @Option(name = "-client-net-ip-address", usage = "Sets the IP Address to listen for connections from clients", required = true)
@@ -26,7 +28,7 @@
     @Option(name = "-client-net-port", usage = "Sets the port to listen for connections from clients (default 1098)")
     public int clientNetPort = 1098;
 
-    @Option(name = "-cluster-net-ip-address", usage = "Sets the IP Address to listen for connections from ", required = true)
+    @Option(name = "-cluster-net-ip-address", usage = "Sets the IP Address to listen for connections from", required = true)
     public String clusterNetIpAddress;
 
     @Option(name = "-cluster-net-port", usage = "Sets the port to listen for connections from node controllers (default 1099)")
@@ -53,9 +55,16 @@
     @Option(name = "-cc-root", usage = "Sets the root folder used for file operations. (default: ClusterControllerService)")
     public String ccRoot = "ClusterControllerService";
 
-    @Option(name = "-cluster-topology", usage = "Sets the XML file that defines the cluster topology. (default: null)")
+    @Option(name = "-cluster-topology", required = false, usage = "Sets the XML file that defines the cluster topology. (default: null)")
     public File clusterTopologyDefinition = null;
 
+    @Option(name = "-app-cc-main-class", required = false, usage = "Application CC Main Class")
+    public String appCCMainClass = null;
+
+    @Argument
+    @Option(name = "--", handler = StopOptionHandler.class)
+    public List<String> appArgs;
+
     public void toCommandLine(List<String> cList) {
         cList.add("-client-net-ip-address");
         cList.add(clientNetIpAddress);
@@ -83,5 +92,15 @@
             cList.add("-cluster-topology");
             cList.add(clusterTopologyDefinition.getAbsolutePath());
         }
+        if (appCCMainClass != null) {
+            cList.add("-app-cc-main-class");
+            cList.add(appCCMainClass);
+        }
+        if (appArgs != null && !appArgs.isEmpty()) {
+            cList.add("--");
+            for (String appArg : appArgs) {
+                cList.add(appArg);
+            }
+        }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index 167eb4b..6a9747b 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -17,7 +17,9 @@
 import java.io.Serializable;
 import java.util.List;
 
+import org.kohsuke.args4j.Argument;
 import org.kohsuke.args4j.Option;
+import org.kohsuke.args4j.spi.StopOptionHandler;
 
 public class NCConfig implements Serializable {
     private static final long serialVersionUID = 1L;
@@ -37,24 +39,28 @@
     @Option(name = "-data-ip-address", usage = "IP Address to bind data listener", required = true)
     public String dataIPAddress;
 
+    @Option(name = "-result-ip-address", usage = "IP Address to bind dataset result distribution listener", required = true)
+    public String datasetIPAddress;
+
     @Option(name = "-iodevices", usage = "Comma separated list of IO Device mount points (default: One device in default temp folder)", required = false)
     public String ioDevices = System.getProperty("java.io.tmpdir");
 
-    @Option(name = "-dcache-client-servers", usage = "Sets the list of DCache servers in the format host1:port1,host2:port2,... (default localhost:54583)")
-    public String dcacheClientServers = "localhost:54583";
-
-    @Option(name = "-dcache-client-server-local", usage = "Sets the local DCache server, if one is available, in the format host:port (default not set)")
-    public String dcacheClientServerLocal;
-
-    @Option(name = "-dcache-client-path", usage = "Sets the path to store the files retrieved from the DCache server (default /tmp/dcache-client)")
-    public String dcacheClientPath = "/tmp/dcache-client";
-
     @Option(name = "-net-thread-count", usage = "Number of threads to use for Network I/O (default: 1)")
     public int nNetThreads = 1;
 
     @Option(name = "-max-memory", usage = "Maximum memory usable at this Node Controller in bytes (default: -1 auto)")
     public int maxMemory = -1;
 
+    @Option(name = "-app-nc-main-class", usage = "Application NC Main Class")
+    public String appNCMainClass;
+
+    @Argument
+    @Option(name = "--", handler = StopOptionHandler.class)
+    public List<String> appArgs;
+
+    @Option(name = "-result-manager-memory", usage = "Memory usable for result caching at this Node Controller in bytes (default: -1 auto)")
+    public int resultManagerMemory = -1;
+
     public void toCommandLine(List<String> cList) {
         cList.add("-cc-host");
         cList.add(ccHost);
@@ -66,19 +72,24 @@
         cList.add(nodeId);
         cList.add("-data-ip-address");
         cList.add(dataIPAddress);
+        cList.add(datasetIPAddress);
         cList.add("-iodevices");
         cList.add(ioDevices);
-        cList.add("-dcache-client-servers");
-        cList.add(dcacheClientServers);
-        if (dcacheClientServerLocal != null) {
-            cList.add("-dcache-client-server-local");
-            cList.add(dcacheClientServerLocal);
-        }
-        cList.add("-dcache-client-path");
-        cList.add(dcacheClientPath);
         cList.add("-net-thread-count");
         cList.add(String.valueOf(nNetThreads));
         cList.add("-max-memory");
         cList.add(String.valueOf(maxMemory));
+        if (appNCMainClass != null) {
+            cList.add("-app-nc-main-class");
+            cList.add(appNCMainClass);
+        }
+        if (appArgs != null && !appArgs.isEmpty()) {
+            cList.add("--");
+            for (String appArg : appArgs) {
+                cList.add(appArg);
+            }
+        }
+        cList.add("-result-manager-memory");
+        cList.add(String.valueOf(resultManagerMemory));
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeParameters.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeParameters.java
index 0161f96..41e0b8d 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeParameters.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeParameters.java
@@ -23,6 +23,8 @@
 
     private ClusterControllerInfo ccInfo;
 
+    private Serializable distributedState;
+
     private int heartbeatPeriod;
 
     private int profileDumpPeriod;
@@ -35,6 +37,14 @@
         this.ccInfo = ccInfo;
     }
 
+    public Serializable getDistributedState() {
+        return distributedState;
+    }
+
+    public void setDistributedState(Serializable distributedState) {
+        this.distributedState = distributedState;
+    }
+
     public int getHeartbeatPeriod() {
         return heartbeatPeriod;
     }
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
index 91cfecf..a897602 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
@@ -33,6 +33,8 @@
 
     private final NetworkAddress dataPort;
 
+    private final NetworkAddress datasetPort;
+
     private final String osName;
 
     private final String arch;
@@ -60,13 +62,14 @@
     private final HeartbeatSchema hbSchema;
 
     public NodeRegistration(InetSocketAddress ncAddress, String nodeId, NCConfig ncConfig, NetworkAddress dataPort,
-            String osName, String arch, String osVersion, int nProcessors, String vmName, String vmVersion,
-            String vmVendor, String classpath, String libraryPath, String bootClasspath, List<String> inputArguments,
-            Map<String, String> systemProperties, HeartbeatSchema hbSchema) {
+            NetworkAddress datasetPort, String osName, String arch, String osVersion, int nProcessors, String vmName,
+            String vmVersion, String vmVendor, String classpath, String libraryPath, String bootClasspath,
+            List<String> inputArguments, Map<String, String> systemProperties, HeartbeatSchema hbSchema) {
         this.ncAddress = ncAddress;
         this.nodeId = nodeId;
         this.ncConfig = ncConfig;
         this.dataPort = dataPort;
+        this.datasetPort = datasetPort;
         this.osName = osName;
         this.arch = arch;
         this.osVersion = osVersion;
@@ -98,6 +101,10 @@
         return dataPort;
     }
 
+    public NetworkAddress getDatasetPort() {
+        return datasetPort;
+    }
+
     public String getOSName() {
         return osName;
     }
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
index 1dba3bc..663c68a 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
@@ -37,6 +37,10 @@
     public long netPayloadBytesWritten;
     public long netSignalingBytesRead;
     public long netSignalingBytesWritten;
+    public long datasetNetPayloadBytesRead;
+    public long datasetNetPayloadBytesWritten;
+    public long datasetNetSignalingBytesRead;
+    public long datasetNetSignalingBytesWritten;
     public long ipcMessagesSent;
     public long ipcMessageBytesSent;
     public long ipcMessagesReceived;
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
index 557a8cb..f6ab9ba 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
@@ -36,11 +36,11 @@
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
 import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
 import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
@@ -68,14 +68,14 @@
         REPORT_PROFILE,
         REGISTER_PARTITION_PROVIDER,
         REGISTER_PARTITION_REQUEST,
-        APPLICATION_STATE_CHANGE_RESPONSE,
+        REGISTER_RESULT_PARTITION_LOCATION,
+        REPORT_RESULT_PARTITION_WRITE_COMPLETION,
+        REPORT_RESULT_PARTITION_FAILURE,
 
         NODE_REGISTRATION_RESULT,
         START_TASKS,
         ABORT_TASKS,
         CLEANUP_JOBLET,
-        CREATE_APPLICATION,
-        DESTROY_APPLICATION,
         REPORT_PARTITION_AVAILABILITY,
         SEND_APPLICATION_MESSAGE,
         GET_NODE_CONTROLLERS_INFO,
@@ -88,7 +88,6 @@
         private static final long serialVersionUID = 1L;
         private byte[] serializedMessage;
         private String nodeId;
-        private String appName;
 
         public String getNodeId() {
             return nodeId;
@@ -102,11 +101,9 @@
             return serializedMessage;
         }
 
-        public SendApplicationMessageFunction(byte[] data, String appName, String nodeId) {
-            super();
+        public SendApplicationMessageFunction(byte[] data, String nodeId) {
             this.serializedMessage = data;
             this.nodeId = nodeId;
-            this.appName = appName;
         }
 
         @Override
@@ -114,10 +111,6 @@
             return FunctionId.SEND_APPLICATION_MESSAGE;
         }
 
-        public String getAppName() {
-            return appName;
-        }
-
     }
 
     public static abstract class Function implements Serializable {
@@ -438,34 +431,124 @@
         }
     }
 
-    public static class ApplicationStateChangeResponseFunction extends Function {
+    public static class RegisterResultPartitionLocationFunction extends Function {
         private static final long serialVersionUID = 1L;
 
-        private final String nodeId;
-        private final String appName;
-        private final ApplicationStatus status;
+        private final JobId jobId;
 
-        public ApplicationStateChangeResponseFunction(String nodeId, String appName, ApplicationStatus status) {
-            this.nodeId = nodeId;
-            this.appName = appName;
-            this.status = status;
+        private final ResultSetId rsId;
+
+        private final boolean orderedResult;
+
+        private final int partition;
+
+        private final int nPartitions;
+
+        private NetworkAddress networkAddress;
+
+        public RegisterResultPartitionLocationFunction(JobId jobId, ResultSetId rsId, boolean orderedResult,
+                int partition, int nPartitions, NetworkAddress networkAddress) {
+            this.jobId = jobId;
+            this.rsId = rsId;
+            this.orderedResult = orderedResult;
+            this.partition = partition;
+            this.nPartitions = nPartitions;
+            this.networkAddress = networkAddress;
         }
 
         @Override
         public FunctionId getFunctionId() {
-            return FunctionId.APPLICATION_STATE_CHANGE_RESPONSE;
+            return FunctionId.REGISTER_RESULT_PARTITION_LOCATION;
         }
 
-        public String getNodeId() {
-            return nodeId;
+        public JobId getJobId() {
+            return jobId;
         }
 
-        public String getApplicationName() {
-            return appName;
+        public ResultSetId getResultSetId() {
+            return rsId;
         }
 
-        public ApplicationStatus getStatus() {
-            return status;
+        public boolean getOrderedResult() {
+            return orderedResult;
+        }
+
+        public int getPartition() {
+            return partition;
+        }
+
+        public int getNPartitions() {
+            return nPartitions;
+        }
+
+        public NetworkAddress getNetworkAddress() {
+            return networkAddress;
+        }
+    }
+
+    public static class ReportResultPartitionWriteCompletionFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+
+        private final ResultSetId rsId;
+
+        private final int partition;
+
+        public ReportResultPartitionWriteCompletionFunction(JobId jobId, ResultSetId rsId, int partition) {
+            this.jobId = jobId;
+            this.rsId = rsId;
+            this.partition = partition;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.REPORT_RESULT_PARTITION_WRITE_COMPLETION;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public ResultSetId getResultSetId() {
+            return rsId;
+        }
+
+        public int getPartition() {
+            return partition;
+        }
+    }
+
+    public static class ReportResultPartitionFailureFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+
+        private final ResultSetId rsId;
+
+        private final int partition;
+
+        public ReportResultPartitionFailureFunction(JobId jobId, ResultSetId rsId, int partition) {
+            this.jobId = jobId;
+            this.rsId = rsId;
+            this.partition = partition;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.REPORT_RESULT_PARTITION_FAILURE;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public ResultSetId getResultSetId() {
+            return rsId;
+        }
+
+        public int getPartition() {
+            return partition;
         }
     }
 
@@ -498,17 +581,14 @@
     public static class StartTasksFunction extends Function {
         private static final long serialVersionUID = 1L;
 
-        private final String appName;
         private final JobId jobId;
         private final byte[] planBytes;
         private final List<TaskAttemptDescriptor> taskDescriptors;
         private final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies;
         private final EnumSet<JobFlag> flags;
 
-        public StartTasksFunction(String appName, JobId jobId, byte[] planBytes,
-                List<TaskAttemptDescriptor> taskDescriptors,
+        public StartTasksFunction(JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
                 Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, EnumSet<JobFlag> flags) {
-            this.appName = appName;
             this.jobId = jobId;
             this.planBytes = planBytes;
             this.taskDescriptors = taskDescriptors;
@@ -521,10 +601,6 @@
             return FunctionId.START_TASKS;
         }
 
-        public String getAppName() {
-            return appName;
-        }
-
         public JobId getJobId() {
             return jobId;
         }
@@ -596,56 +672,6 @@
         }
     }
 
-    public static class CreateApplicationFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final String appName;
-        private final boolean deployHar;
-        private final byte[] serializedDistributedState;
-
-        public CreateApplicationFunction(String appName, boolean deployHar, byte[] serializedDistributedState) {
-            this.appName = appName;
-            this.deployHar = deployHar;
-            this.serializedDistributedState = serializedDistributedState;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.CREATE_APPLICATION;
-        }
-
-        public String getAppName() {
-            return appName;
-        }
-
-        public boolean isDeployHar() {
-            return deployHar;
-        }
-
-        public byte[] getSerializedDistributedState() {
-            return serializedDistributedState;
-        }
-    }
-
-    public static class DestroyApplicationFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final String appName;
-
-        public DestroyApplicationFunction(String appName) {
-            this.appName = appName;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.DESTROY_APPLICATION;
-        }
-
-        public String getAppName() {
-            return appName;
-        }
-    }
-
     public static class GetNodeControllersInfoFunction extends Function {
         private static final long serialVersionUID = 1L;
 
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
index bbaab4e..057a0f4 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
@@ -16,9 +16,10 @@
 
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
 import edu.uci.ics.hyracks.control.common.base.IClusterController;
 import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
@@ -95,16 +96,29 @@
     }
 
     @Override
-    public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception {
-        CCNCFunctions.ApplicationStateChangeResponseFunction fn = new CCNCFunctions.ApplicationStateChangeResponseFunction(
-                nodeId, appName, status);
+    public void sendApplicationMessageToCC(byte[] data, String nodeId) throws Exception {
+        CCNCFunctions.SendApplicationMessageFunction fn = new CCNCFunctions.SendApplicationMessageFunction(data, nodeId);
+        ipcHandle.send(-1, fn, null);
+    }
+
+    public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
+            int nPartitions, NetworkAddress networkAddress) throws Exception {
+        CCNCFunctions.RegisterResultPartitionLocationFunction fn = new CCNCFunctions.RegisterResultPartitionLocationFunction(
+                jobId, rsId, orderedResult, partition, nPartitions, networkAddress);
         ipcHandle.send(-1, fn, null);
     }
 
     @Override
-    public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception {
-        CCNCFunctions.SendApplicationMessageFunction fn = new CCNCFunctions.SendApplicationMessageFunction(data,
-                appName, nodeId);
+    public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws Exception {
+        CCNCFunctions.ReportResultPartitionWriteCompletionFunction fn = new CCNCFunctions.ReportResultPartitionWriteCompletionFunction(
+                jobId, rsId, partition);
+        ipcHandle.send(-1, fn, null);
+    }
+
+    @Override
+    public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws Exception {
+        CCNCFunctions.ReportResultPartitionFailureFunction fn = new CCNCFunctions.ReportResultPartitionFailureFunction(
+                jobId, rsId, partition);
         ipcHandle.send(-1, fn, null);
     }
 
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
index 10c0a7c..e4355aa 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
@@ -38,10 +38,10 @@
     }
 
     @Override
-    public void startTasks(String appName, JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
+    public void startTasks(JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
             Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, EnumSet<JobFlag> flags) throws Exception {
-        CCNCFunctions.StartTasksFunction stf = new CCNCFunctions.StartTasksFunction(appName, jobId, planBytes,
-                taskDescriptors, connectorPolicies, flags);
+        CCNCFunctions.StartTasksFunction stf = new CCNCFunctions.StartTasksFunction(jobId, planBytes, taskDescriptors,
+                connectorPolicies, flags);
         ipcHandle.send(-1, stf, null);
     }
 
@@ -58,20 +58,6 @@
     }
 
     @Override
-    public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
-            throws Exception {
-        CCNCFunctions.CreateApplicationFunction caf = new CCNCFunctions.CreateApplicationFunction(appName, deployHar,
-                serializedDistributedState);
-        ipcHandle.send(-1, caf, null);
-    }
-
-    @Override
-    public void destroyApplication(String appName) throws Exception {
-        CCNCFunctions.DestroyApplicationFunction daf = new CCNCFunctions.DestroyApplicationFunction(appName);
-        ipcHandle.send(-1, daf, null);
-    }
-
-    @Override
     public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception {
         CCNCFunctions.ReportPartitionAvailabilityFunction rpaf = new CCNCFunctions.ReportPartitionAvailabilityFunction(
                 pid, networkAddress);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/pom.xml b/hyracks/hyracks-control/hyracks-control-nc/pom.xml
index 6a7e5d8..e163d2b 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/pom.xml
+++ b/hyracks/hyracks-control/hyracks-control-nc/pom.xml
@@ -24,12 +24,6 @@
   </build>
   <dependencies>
   	<dependency>
-  		<groupId>edu.uci.ics.dcache</groupId>
-  		<artifactId>dcache-client</artifactId>
-  		<version>0.0.1</version>
-  		<scope>compile</scope>
-  	</dependency>
-  	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-common</artifactId>
   		<version>0.2.3-SNAPSHOT</version>
@@ -41,6 +35,11 @@
   		<artifactId>hyracks-net</artifactId>
   		<version>0.2.3-SNAPSHOT</version>
   	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-comm</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  	</dependency>
   </dependencies>
   <reporting>
     <plugins>
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java
index dde7abc..d31d09e 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java
@@ -16,8 +16,6 @@
 
 import org.kohsuke.args4j.CmdLineParser;
 
-import edu.uci.ics.dcache.client.DCacheClient;
-import edu.uci.ics.dcache.client.DCacheClientConfig;
 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 
 public class NCDriver {
@@ -32,13 +30,6 @@
             return;
         }
 
-        DCacheClientConfig dccConfig = new DCacheClientConfig();
-        dccConfig.servers = ncConfig.dcacheClientServers;
-        dccConfig.serverLocal = ncConfig.dcacheClientServerLocal;
-        dccConfig.path = ncConfig.dcacheClientPath;
-
-        DCacheClient.get().init(dccConfig);
-
         final NodeControllerService nService = new NodeControllerService(ncConfig);
         nService.start();
         Runtime.getRuntime().addShutdownHook(new Thread() {
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 0195143..10e0dba 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -43,8 +43,10 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 
+import edu.uci.ics.hyracks.api.application.INCApplicationEntryPoint;
 import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
 import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
@@ -61,7 +63,9 @@
 import edu.uci.ics.hyracks.control.common.work.FutureValue;
 import edu.uci.ics.hyracks.control.common.work.WorkQueue;
 import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
+import edu.uci.ics.hyracks.control.nc.dataset.DatasetPartitionManager;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.net.DatasetNetworkManager;
 import edu.uci.ics.hyracks.control.nc.net.NetworkManager;
 import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
 import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
@@ -69,8 +73,6 @@
 import edu.uci.ics.hyracks.control.nc.work.ApplicationMessageWork;
 import edu.uci.ics.hyracks.control.nc.work.BuildJobProfilesWork;
 import edu.uci.ics.hyracks.control.nc.work.CleanupJobletWork;
-import edu.uci.ics.hyracks.control.nc.work.CreateApplicationWork;
-import edu.uci.ics.hyracks.control.nc.work.DestroyApplicationWork;
 import edu.uci.ics.hyracks.control.nc.work.ReportPartitionAvailabilityWork;
 import edu.uci.ics.hyracks.control.nc.work.StartTasksWork;
 import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
@@ -94,6 +96,10 @@
 
     private final NetworkManager netManager;
 
+    private final IDatasetPartitionManager datasetPartitionManager;
+
+    private final DatasetNetworkManager datasetNetworkManager;
+
     private final WorkQueue queue;
 
     private final Timer timer;
@@ -114,7 +120,9 @@
 
     private final ServerContext serverCtx;
 
-    private final Map<String, NCApplicationContext> applications;
+    private NCApplicationContext appCtx;
+
+    private INCApplicationEntryPoint ncAppEntryPoint;
 
     private final MemoryMXBean memoryMXBean;
 
@@ -140,14 +148,17 @@
             throw new Exception("id not set");
         }
         partitionManager = new PartitionManager(this);
-        netManager = new NetworkManager(getIpAddress(ncConfig), partitionManager, ncConfig.nNetThreads);
+        netManager = new NetworkManager(getIpAddress(ncConfig.dataIPAddress), partitionManager, ncConfig.nNetThreads);
+
+        datasetPartitionManager = new DatasetPartitionManager(this, executor, ncConfig.resultManagerMemory);
+        datasetNetworkManager = new DatasetNetworkManager(getIpAddress(ncConfig.datasetIPAddress),
+                datasetPartitionManager, ncConfig.nNetThreads);
 
         queue = new WorkQueue();
         jobletMap = new Hashtable<JobId, Joblet>();
         timer = new Timer(true);
         serverCtx = new ServerContext(ServerContext.ServerType.NODE_CONTROLLER, new File(new File(
                 NodeControllerService.class.getName()), id));
-        applications = new Hashtable<String, NCApplicationContext>();
         memoryMXBean = ManagementFactory.getMemoryMXBean();
         gcMXBeans = ManagementFactory.getGarbageCollectorMXBeans();
         threadMXBean = ManagementFactory.getThreadMXBean();
@@ -161,6 +172,10 @@
         return ctx;
     }
 
+    public NCApplicationContext getApplicationContext() {
+        return appCtx;
+    }
+
     private static List<IODeviceHandle> getDevices(String ioDevices) {
         List<IODeviceHandle> devices = new ArrayList<IODeviceHandle>();
         StringTokenizer tok = new StringTokenizer(ioDevices, ",");
@@ -205,6 +220,10 @@
         LOGGER.log(Level.INFO, "Starting NodeControllerService");
         ipc.start();
         netManager.start();
+
+        startApplication();
+
+        datasetNetworkManager.start();
         IIPCHandle ccIPCHandle = ipc.getHandle(new InetSocketAddress(ncConfig.ccHost, ncConfig.ccPort));
         this.ccs = new ClusterControllerRemoteProxy(ccIPCHandle);
         HeartbeatSchema.GarbageCollectorInfo[] gcInfos = new HeartbeatSchema.GarbageCollectorInfo[gcMXBeans.size()];
@@ -213,10 +232,11 @@
         }
         HeartbeatSchema hbSchema = new HeartbeatSchema(gcInfos);
         ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig, netManager.getNetworkAddress(),
-                osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), osMXBean.getAvailableProcessors(),
-                runtimeMXBean.getVmName(), runtimeMXBean.getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean
-                        .getClassPath(), runtimeMXBean.getLibraryPath(), runtimeMXBean.getBootClassPath(),
-                runtimeMXBean.getInputArguments(), runtimeMXBean.getSystemProperties(), hbSchema));
+                datasetNetworkManager.getNetworkAddress(), osMXBean.getName(), osMXBean.getArch(), osMXBean
+                        .getVersion(), osMXBean.getAvailableProcessors(), runtimeMXBean.getVmName(), runtimeMXBean
+                        .getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean.getClassPath(), runtimeMXBean
+                        .getLibraryPath(), runtimeMXBean.getBootClassPath(), runtimeMXBean.getInputArguments(),
+                runtimeMXBean.getSystemProperties(), hbSchema));
 
         synchronized (this) {
             while (registrationPending) {
@@ -226,6 +246,7 @@
         if (registrationException != null) {
             throw registrationException;
         }
+        appCtx.setDistributedState(nodeParameters.getDistributedState());
 
         queue.start();
 
@@ -240,6 +261,21 @@
         }
 
         LOGGER.log(Level.INFO, "Started NodeControllerService");
+        if (ncAppEntryPoint != null) {
+            ncAppEntryPoint.notifyStartupComplete();
+        }
+    }
+
+    private void startApplication() throws Exception {
+        appCtx = new NCApplicationContext(serverCtx, ctx, id);
+        String className = ncConfig.appNCMainClass;
+        if (className != null) {
+            Class<?> c = Class.forName(className);
+            ncAppEntryPoint = (INCApplicationEntryPoint) c.newInstance();
+            String[] args = ncConfig.appArgs == null ? new String[0] : ncConfig.appArgs
+                    .toArray(new String[ncConfig.appArgs.size()]);
+            ncAppEntryPoint.start(appCtx, args);
+        }
     }
 
     @Override
@@ -247,8 +283,10 @@
         LOGGER.log(Level.INFO, "Stopping NodeControllerService");
         executor.shutdownNow();
         partitionManager.close();
+        datasetPartitionManager.close();
         heartbeatTask.cancel();
         netManager.stop();
+        datasetNetworkManager.stop();
         queue.stop();
         LOGGER.log(Level.INFO, "Stopped NodeControllerService");
     }
@@ -261,10 +299,6 @@
         return serverCtx;
     }
 
-    public Map<String, NCApplicationContext> getApplications() {
-        return applications;
-    }
-
     public Map<JobId, Joblet> getJobletMap() {
         return jobletMap;
     }
@@ -273,6 +307,10 @@
         return netManager;
     }
 
+    public DatasetNetworkManager getDatasetNetworkManager() {
+        return datasetNetworkManager;
+    }
+
     public PartitionManager getPartitionManager() {
         return partitionManager;
     }
@@ -297,8 +335,7 @@
         return queue;
     }
 
-    private static InetAddress getIpAddress(NCConfig ncConfig) throws Exception {
-        String ipaddrStr = ncConfig.dataIPAddress;
+    private static InetAddress getIpAddress(String ipaddrStr) throws Exception {
         ipaddrStr = ipaddrStr.trim();
         Pattern pattern = Pattern.compile("(\\d{1,3})\\.(\\d{1,3})\\.(\\d{1,3})\\.(\\d{1,3})");
         Matcher m = pattern.matcher(ipaddrStr);
@@ -355,6 +392,12 @@
             hbData.netSignalingBytesRead = netPC.getSignalingBytesRead();
             hbData.netSignalingBytesWritten = netPC.getSignalingBytesWritten();
 
+            MuxDemuxPerformanceCounters datasetNetPC = datasetNetworkManager.getPerformanceCounters();
+            hbData.datasetNetPayloadBytesRead = datasetNetPC.getPayloadBytesRead();
+            hbData.datasetNetPayloadBytesWritten = datasetNetPC.getPayloadBytesWritten();
+            hbData.datasetNetSignalingBytesRead = datasetNetPC.getSignalingBytesRead();
+            hbData.datasetNetSignalingBytesWritten = datasetNetPC.getSignalingBytesWritten();
+
             IPCPerformanceCounters ipcPC = ipc.getPerformanceCounters();
             hbData.ipcMessagesSent = ipcPC.getMessageSentCount();
             hbData.ipcMessageBytesSent = ipcPC.getMessageBytesSent();
@@ -400,13 +443,13 @@
                 case SEND_APPLICATION_MESSAGE: {
                     CCNCFunctions.SendApplicationMessageFunction amf = (CCNCFunctions.SendApplicationMessageFunction) fn;
                     queue.schedule(new ApplicationMessageWork(NodeControllerService.this, amf.getMessage(), amf
-                            .getAppName(), amf.getNodeId()));
+                            .getNodeId()));
                     return;
                 }
                 case START_TASKS: {
                     CCNCFunctions.StartTasksFunction stf = (CCNCFunctions.StartTasksFunction) fn;
-                    queue.schedule(new StartTasksWork(NodeControllerService.this, stf.getAppName(), stf.getJobId(), stf
-                            .getPlanBytes(), stf.getTaskDescriptors(), stf.getConnectorPolicies(), stf.getFlags()));
+                    queue.schedule(new StartTasksWork(NodeControllerService.this, stf.getJobId(), stf.getPlanBytes(),
+                            stf.getTaskDescriptors(), stf.getConnectorPolicies(), stf.getFlags()));
                     return;
                 }
 
@@ -422,19 +465,6 @@
                     return;
                 }
 
-                case CREATE_APPLICATION: {
-                    CCNCFunctions.CreateApplicationFunction caf = (CCNCFunctions.CreateApplicationFunction) fn;
-                    queue.schedule(new CreateApplicationWork(NodeControllerService.this, caf.getAppName(), caf
-                            .isDeployHar(), caf.getSerializedDistributedState()));
-                    return;
-                }
-
-                case DESTROY_APPLICATION: {
-                    CCNCFunctions.DestroyApplicationFunction daf = (CCNCFunctions.DestroyApplicationFunction) fn;
-                    queue.schedule(new DestroyApplicationWork(NodeControllerService.this, daf.getAppName()));
-                    return;
-                }
-
                 case REPORT_PARTITION_AVAILABILITY: {
                     CCNCFunctions.ReportPartitionAvailabilityFunction rpaf = (CCNCFunctions.ReportPartitionAvailabilityFunction) fn;
                     queue.schedule(new ReportPartitionAvailabilityWork(NodeControllerService.this, rpaf
@@ -459,7 +489,11 @@
         }
     }
 
-    public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception {
-        ccs.sendApplicationMessageToCC(data, appName, nodeId);
+    public void sendApplicationMessageToCC(byte[] data, String nodeId) throws Exception {
+        ccs.sendApplicationMessageToCC(data, nodeId);
+    }
+
+    public IDatasetPartitionManager getDatasetPartitionManager() {
+        return datasetPartitionManager;
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index eba3ec9..d6ea111 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -34,6 +34,7 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -348,8 +349,12 @@
     }
 
     @Override
+    public IDatasetPartitionManager getDatasetPartitionManager() {
+        return ncs.getDatasetPartitionManager();
+    }
+
+    @Override
     public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception {
-        this.ncs.sendApplicationMessageToCC(message, this.getJobletContext().getApplicationContext()
-                .getApplicationName(), nodeId);
+        this.ncs.sendApplicationMessageToCC(message, nodeId);
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
index 1121c6c..3e07fb5 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
@@ -4,7 +4,6 @@
 import java.io.Serializable;
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
-import edu.uci.ics.hyracks.api.application.INCBootstrap;
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
@@ -14,9 +13,8 @@
     private final IHyracksRootContext rootCtx;
     private Object appObject;
 
-    public NCApplicationContext(ServerContext serverCtx, IHyracksRootContext rootCtx, String appName, String nodeId)
-            throws IOException {
-        super(serverCtx, appName);
+    public NCApplicationContext(ServerContext serverCtx, IHyracksRootContext rootCtx, String nodeId) throws IOException {
+        super(serverCtx);
         this.nodeId = nodeId;
         this.rootCtx = rootCtx;
     }
@@ -31,19 +29,6 @@
     }
 
     @Override
-    protected void start() throws Exception {
-        ((INCBootstrap) bootstrap).setApplicationContext(this);
-        bootstrap.start();
-    }
-
-    @Override
-    protected void stop() throws Exception {
-        if (bootstrap != null) {
-            bootstrap.stop();
-        }
-    }
-
-    @Override
     public IHyracksRootContext getRootContext() {
         return rootCtx;
     }
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java
new file mode 100644
index 0000000..cecd677
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java
@@ -0,0 +1,237 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.dataset;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionWriter;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
+
+public class DatasetMemoryManager {
+    private final Set<Page> availPages;
+
+    private final LeastRecentlyUsedList leastRecentlyUsedList;
+
+    private final Map<ResultSetPartitionId, PartitionNode> resultPartitionNodesMap;
+
+    private final static int FRAME_SIZE = 32768;
+
+    public DatasetMemoryManager(int availableMemory) {
+        availPages = new HashSet<Page>();
+
+        // Atleast have one page for temporarily storing the results.
+        if (availableMemory <= 0)
+            availableMemory = FRAME_SIZE;
+
+        while (availableMemory >= FRAME_SIZE) {
+            /* TODO(madhusudancs): Should we have some way of accounting this memory usage by using Hyrack's allocateFrame()
+             * instead of direct ByteBuffer.allocate()?
+             */
+            availPages.add(new Page(ByteBuffer.allocate(FRAME_SIZE)));
+            availableMemory -= FRAME_SIZE;
+        }
+
+        leastRecentlyUsedList = new LeastRecentlyUsedList();
+        resultPartitionNodesMap = new HashMap<ResultSetPartitionId, PartitionNode>();
+    }
+
+    public Page requestPage(ResultSetPartitionId resultSetPartitionId, IDatasetPartitionWriter dpw)
+            throws OutOfMemoryError, HyracksDataException {
+        Page page;
+        if (availPages.isEmpty()) {
+            page = evictPage();
+        } else {
+            page = getAvailablePage();
+        }
+
+        page.clear();
+
+        /*
+         * It is extremely important to update the reference after obtaining the page because, in the cases where memory
+         * manager is allocated only one page of memory, the front of the LRU list should not be created by the
+         * update reference call before a page is pushed on to the element of the LRU list. So we first obtain the page,
+         * then make a updateReference call which in turn creates a new node in the LRU list and then add the page to it.
+         */
+        PartitionNode pn = updateReference(resultSetPartitionId, dpw);
+        pn.add(page);
+        return page;
+    }
+
+    public void pageReferenced(ResultSetPartitionId resultSetPartitionId) {
+        // When a page is referenced the dataset partition writer should already be known, so we pass null.
+        updateReference(resultSetPartitionId, null);
+    }
+
+    public int getPageSize() {
+        return FRAME_SIZE;
+    }
+
+    protected void insertPartitionNode(ResultSetPartitionId resultSetPartitionId, PartitionNode pn) {
+        leastRecentlyUsedList.add(pn);
+        resultPartitionNodesMap.put(resultSetPartitionId, pn);
+    }
+
+    protected synchronized PartitionNode updateReference(ResultSetPartitionId resultSetPartitionId,
+            IDatasetPartitionWriter dpw) {
+        PartitionNode pn = null;
+
+        if (!resultPartitionNodesMap.containsKey(resultSetPartitionId)) {
+            if (dpw != null) {
+                pn = new PartitionNode(resultSetPartitionId, dpw);
+                insertPartitionNode(resultSetPartitionId, pn);
+            }
+            return pn;
+        }
+        pn = resultPartitionNodesMap.get(resultSetPartitionId);
+        leastRecentlyUsedList.remove(pn);
+        insertPartitionNode(resultSetPartitionId, pn);
+
+        return pn;
+    }
+
+    protected synchronized Page evictPage() throws HyracksDataException {
+        PartitionNode pn = leastRecentlyUsedList.getFirst();
+        IDatasetPartitionWriter dpw = pn.getDatasetPartitionWriter();
+        Page page = dpw.returnPage();
+
+        /* If the partition holding the pages breaks the contract by not returning the page or it has no page, just take
+         * away all the pages allocated to it and add to the available pages set.
+         */
+        if (page == null) {
+            availPages.addAll(pn);
+            pn.clear();
+            resultPartitionNodesMap.remove(pn.getResultSetPartitionId());
+            leastRecentlyUsedList.remove(pn);
+
+            /* Based on the assumption that if the dataset partition writer returned a null page, it should be lying about
+             * the number of pages it holds in which case we just evict all the pages it holds and should thus be able to
+             * add all those pages to available set and we have at least one page to allocate back.
+             */
+            page = getAvailablePage();
+        } else {
+            pn.remove(page);
+
+            // If the partition no more holds any pages, remove it from the linked list and the hash map.
+            if (pn.isEmpty()) {
+                resultPartitionNodesMap.remove(pn.getResultSetPartitionId());
+                leastRecentlyUsedList.remove(pn);
+            }
+        }
+
+        return page;
+    }
+
+    protected synchronized Page getAvailablePage() {
+        Iterator<Page> iter = availPages.iterator();
+        Page page = iter.next();
+        iter.remove();
+        return page;
+    }
+
+    private class LeastRecentlyUsedList {
+        private PartitionNode head;
+
+        private PartitionNode tail;
+
+        public LeastRecentlyUsedList() {
+            head = null;
+            tail = null;
+        }
+
+        public void add(PartitionNode node) {
+            if (head == null) {
+                head = tail = node;
+                return;
+            }
+            tail.setNext(node);
+            node.setPrev(tail);
+            tail = node;
+        }
+
+        public void remove(PartitionNode node) {
+            if ((node == head) && (node == tail)) {
+                head = tail = null;
+                return;
+            } else if (node == head) {
+                head = head.getNext();
+                head.setPrev(null);
+                return;
+            } else if (node == tail) {
+                tail = tail.getPrev();
+                tail.setNext(null);
+                return;
+            } else {
+                PartitionNode prev = node.getPrev();
+                PartitionNode next = node.getNext();
+                prev.setNext(next);
+                next.setPrev(prev);
+            }
+        }
+
+        public PartitionNode getFirst() {
+            return head;
+        }
+    }
+
+    private class PartitionNode extends HashSet<Page> {
+        private static final long serialVersionUID = 1L;
+
+        private final ResultSetPartitionId resultSetPartitionId;
+
+        private final IDatasetPartitionWriter datasetPartitionWriter;
+
+        private PartitionNode prev;
+
+        private PartitionNode next;
+
+        public PartitionNode(ResultSetPartitionId resultSetPartitionId, IDatasetPartitionWriter datasetPartitionWriter) {
+            this.resultSetPartitionId = resultSetPartitionId;
+            this.datasetPartitionWriter = datasetPartitionWriter;
+            prev = null;
+            next = null;
+        }
+
+        public ResultSetPartitionId getResultSetPartitionId() {
+            return resultSetPartitionId;
+        }
+
+        public IDatasetPartitionWriter getDatasetPartitionWriter() {
+            return datasetPartitionWriter;
+        }
+
+        public void setPrev(PartitionNode node) {
+            prev = node;
+        }
+
+        public PartitionNode getPrev() {
+            return prev;
+        }
+
+        public void setNext(PartitionNode node) {
+            next = node;
+        }
+
+        public PartitionNode getNext() {
+            return next;
+        }
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java
new file mode 100644
index 0000000..1cad54b
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java
@@ -0,0 +1,124 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.dataset;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
+
+public class DatasetPartitionManager implements IDatasetPartitionManager {
+    private final NodeControllerService ncs;
+
+    private final Executor executor;
+
+    private final Map<JobId, ResultState[]> partitionResultStateMap;
+
+    private final DefaultDeallocatableRegistry deallocatableRegistry;
+
+    private final IWorkspaceFileFactory fileFactory;
+
+    private final DatasetMemoryManager datasetMemoryManager;
+
+    public DatasetPartitionManager(NodeControllerService ncs, Executor executor, int availableMemory) {
+        this.ncs = ncs;
+        this.executor = executor;
+        partitionResultStateMap = new HashMap<JobId, ResultState[]>();
+        deallocatableRegistry = new DefaultDeallocatableRegistry();
+        fileFactory = new WorkspaceFileFactory(deallocatableRegistry, (IOManager) ncs.getRootContext().getIOManager());
+        datasetMemoryManager = new DatasetMemoryManager(availableMemory);
+    }
+
+    @Override
+    public IFrameWriter createDatasetPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
+            int partition, int nPartitions) throws HyracksException {
+        DatasetPartitionWriter dpw = null;
+        JobId jobId = ctx.getJobletContext().getJobId();
+        try {
+            ncs.getClusterController().registerResultPartitionLocation(jobId, rsId, orderedResult, partition,
+                    nPartitions, ncs.getDatasetNetworkManager().getNetworkAddress());
+            dpw = new DatasetPartitionWriter(ctx, this, jobId, rsId, partition, datasetMemoryManager);
+
+            ResultState[] resultStates = partitionResultStateMap.get(jobId);
+            if (resultStates == null) {
+                resultStates = new ResultState[nPartitions];
+                partitionResultStateMap.put(jobId, resultStates);
+            }
+            resultStates[partition] = dpw.getResultState();
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+
+        return dpw;
+    }
+
+    @Override
+    public void reportPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws HyracksException {
+        try {
+            ncs.getClusterController().reportResultPartitionWriteCompletion(jobId, rsId, partition);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    @Override
+    public void reportPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws HyracksException {
+        try {
+            ncs.getClusterController().reportResultPartitionFailure(jobId, rsId, partition);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    @Override
+    public void initializeDatasetPartitionReader(JobId jobId, int partition, IFrameWriter writer)
+            throws HyracksException {
+        ResultState[] resultStates = partitionResultStateMap.get(jobId);
+
+        if (resultStates == null) {
+            throw new HyracksException("Unknown JobId " + jobId);
+        }
+
+        ResultState resultState = resultStates[partition];
+        if (resultState == null) {
+            throw new HyracksException("No DatasetPartitionWriter for partition " + partition);
+        }
+
+        IDatasetPartitionReader dpr = new DatasetPartitionReader(datasetMemoryManager, executor, resultState);
+        dpr.writeTo(writer);
+    }
+
+    @Override
+    public IWorkspaceFileFactory getFileFactory() {
+        return fileFactory;
+    }
+
+    @Override
+    public void close() {
+        deallocatableRegistry.close();
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java
new file mode 100644
index 0000000..296c502
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.dataset;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.Executor;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionReader;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.IFileHandle;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
+
+public class DatasetPartitionReader implements IDatasetPartitionReader {
+    private static final Logger LOGGER = Logger.getLogger(DatasetPartitionReader.class.getName());
+
+    private final DatasetMemoryManager datasetMemoryManager;
+
+    private final Executor executor;
+
+    private final ResultState resultState;
+
+    private IFileHandle fileHandle;
+
+    public DatasetPartitionReader(DatasetMemoryManager datasetMemoryManager, Executor executor, ResultState resultState) {
+        this.datasetMemoryManager = datasetMemoryManager;
+        this.executor = executor;
+        this.resultState = resultState;
+    }
+
+    private long read(long offset, ByteBuffer buffer) throws HyracksDataException {
+        long readSize = 0;
+        synchronized (resultState) {
+            while (offset >= resultState.getSize() && !resultState.getEOS()) {
+                try {
+                    resultState.wait();
+                } catch (InterruptedException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        }
+
+        if (offset >= resultState.getSize() && resultState.getEOS()) {
+            return readSize;
+        }
+
+        if (offset < resultState.getPersistentSize()) {
+            readSize = resultState.getIOManager().syncRead(fileHandle, offset, buffer);
+        }
+
+        if (readSize < buffer.capacity()) {
+            long localPageOffset = offset - resultState.getPersistentSize();
+            int localPageIndex = (int) (localPageOffset / datasetMemoryManager.getPageSize());
+            int pageOffset = (int) (localPageOffset % datasetMemoryManager.getPageSize());
+            Page page = resultState.getPage(localPageIndex);
+            readSize += buffer.remaining();
+            buffer.put(page.getBuffer().array(), pageOffset, buffer.remaining());
+        }
+
+        datasetMemoryManager.pageReferenced(resultState.getResultSetPartitionId());
+        return readSize;
+    }
+
+    @Override
+    public void writeTo(final IFrameWriter writer) {
+        executor.execute(new Runnable() {
+            @Override
+            public void run() {
+                NetworkOutputChannel channel = (NetworkOutputChannel) writer;
+                channel.setFrameSize(resultState.getFrameSize());
+                try {
+                    fileHandle = resultState.getIOManager().open(resultState.getValidFileReference(),
+                            IIOManager.FileReadWriteMode.READ_ONLY, IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+                    channel.open();
+                    try {
+                        long offset = 0;
+                        ByteBuffer buffer = ByteBuffer.allocate(resultState.getFrameSize());
+                        while (true) {
+                            buffer.clear();
+                            long size = read(offset, buffer);
+                            if (size <= 0) {
+                                break;
+                            } else if (size < buffer.limit()) {
+                                throw new HyracksDataException("Premature end of file - readSize: " + size
+                                        + " buffer limit: " + buffer.limit());
+                            }
+                            offset += size;
+                            buffer.flip();
+                            channel.nextFrame(buffer);
+                        }
+                    } finally {
+                        channel.close();
+                        resultState.getIOManager().close(fileHandle);
+                    }
+                } catch (InterruptedException e) {
+                    throw new RuntimeException(e);
+                } catch (HyracksDataException e) {
+                    throw new RuntimeException(e);
+                }
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("result reading successful(" + resultState.getResultSetPartitionId() + ")");
+                }
+            }
+        });
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
new file mode 100644
index 0000000..f6ae540
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
@@ -0,0 +1,148 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.dataset;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionWriter;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IFileHandle;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
+
+public class DatasetPartitionWriter implements IDatasetPartitionWriter {
+    private static final Logger LOGGER = Logger.getLogger(DatasetPartitionWriter.class.getName());
+
+    private static final String FILE_PREFIX = "result_";
+
+    private final IDatasetPartitionManager manager;
+
+    private final JobId jobId;
+
+    private final ResultSetId resultSetId;
+
+    private final int partition;
+
+    private final DatasetMemoryManager datasetMemoryManager;
+
+    private final ResultSetPartitionId resultSetPartitionId;
+
+    private final ResultState resultState;
+
+    private IFileHandle fileHandle;
+
+    public DatasetPartitionWriter(IHyracksTaskContext ctx, IDatasetPartitionManager manager, JobId jobId,
+            ResultSetId rsId, int partition, DatasetMemoryManager datasetMemoryManager) {
+        this.manager = manager;
+        this.jobId = jobId;
+        this.resultSetId = rsId;
+        this.partition = partition;
+        this.datasetMemoryManager = datasetMemoryManager;
+
+        resultSetPartitionId = new ResultSetPartitionId(jobId, rsId, partition);
+        resultState = new ResultState(resultSetPartitionId, ctx.getIOManager(), ctx.getFrameSize());
+    }
+
+    public ResultState getResultState() {
+        return resultState;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("open(" + partition + ")");
+        }
+        String fName = FILE_PREFIX + String.valueOf(partition);
+        FileReference fRef = manager.getFileFactory().createUnmanagedWorkspaceFile(fName);
+        fileHandle = resultState.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+        resultState.init(fRef);
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        int srcOffset = 0;
+        Page destPage = resultState.getLastPage();
+
+        while (srcOffset < buffer.limit()) {
+            if ((destPage == null) || (destPage.getBuffer().remaining() <= 0)) {
+                destPage = datasetMemoryManager.requestPage(resultSetPartitionId, this);
+                resultState.addPage(destPage);
+            }
+            int srcLength = Math.min(buffer.limit() - srcOffset, destPage.getBuffer().remaining());
+            destPage.getBuffer().put(buffer.array(), srcOffset, srcLength);
+            srcOffset += srcLength;
+            resultState.incrementSize(srcLength);
+        }
+
+        synchronized (resultState) {
+            resultState.notifyAll();
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        try {
+            manager.reportPartitionFailure(jobId, resultSetId, partition);
+        } catch (HyracksException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("close(" + partition + ")");
+        }
+
+        try {
+            synchronized (resultState) {
+                resultState.setEOS(true);
+                resultState.notifyAll();
+            }
+            manager.reportPartitionWriteCompletion(jobId, resultSetId, partition);
+        } catch (HyracksException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public Page returnPage() throws HyracksDataException {
+        Page page = resultState.removePage(0);
+
+        IIOManager ioManager = resultState.getIOManager();
+
+        // If we do not have any pages to be given back close the write channel since we don't write any more, return null.
+        if (page == null) {
+            ioManager.close(fileHandle);
+            return null;
+        }
+
+        page.getBuffer().flip();
+
+        long delta = ioManager.syncWrite(fileHandle, resultState.getPersistentSize(), page.getBuffer());
+        resultState.incrementPersistentSize(delta);
+        return page;
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
new file mode 100644
index 0000000..3db3fd9
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
@@ -0,0 +1,173 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.dataset;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
+
+public class ResultState implements IStateObject {
+    private final ResultSetPartitionId resultSetPartitionId;
+
+    private final int frameSize;
+
+    private final IIOManager ioManager;
+
+    private final AtomicBoolean eos;
+
+    private final AtomicBoolean readEOS;
+
+    private final List<Page> localPageList;
+
+    private FileReference fileRef;
+
+    private long size;
+
+    private long persistentSize;
+
+    ResultState(ResultSetPartitionId resultSetPartitionId, IIOManager ioManager, int frameSize) {
+        this.resultSetPartitionId = resultSetPartitionId;
+        this.ioManager = ioManager;
+        this.frameSize = frameSize;
+        eos = new AtomicBoolean(false);
+        readEOS = new AtomicBoolean(false);
+        localPageList = new ArrayList<Page>();
+    }
+
+    public synchronized void init(FileReference fileRef) {
+        this.fileRef = fileRef;
+
+        size = 0;
+        persistentSize = 0;
+        notifyAll();
+    }
+
+    public ResultSetPartitionId getResultSetPartitionId() {
+        return resultSetPartitionId;
+    }
+
+    public int getFrameSize() {
+        return frameSize;
+    }
+
+    public IIOManager getIOManager() {
+        return ioManager;
+    }
+
+    public synchronized void incrementSize(long delta) {
+        size += delta;
+    }
+
+    public synchronized long getSize() {
+        return size;
+    }
+
+    public synchronized void incrementPersistentSize(long delta) {
+        persistentSize += delta;
+    }
+
+    public synchronized long getPersistentSize() {
+        return persistentSize;
+    }
+
+    public void setEOS(boolean eos) {
+        this.eos.set(eos);
+    }
+
+    public boolean getEOS() {
+        return eos.get();
+    }
+
+    public boolean getReadEOS() {
+        return readEOS.get();
+    }
+
+    public synchronized void addPage(Page page) {
+        localPageList.add(page);
+    }
+
+    public synchronized Page removePage(int index) {
+        Page page = null;
+        if (!localPageList.isEmpty()) {
+            page = localPageList.remove(index);
+        }
+        return page;
+    }
+
+    public synchronized Page getPage(int index) {
+        Page page = null;
+        if (!localPageList.isEmpty()) {
+            page = localPageList.get(index);
+        }
+        return page;
+    }
+
+    public synchronized Page getLastPage() {
+        Page page = null;
+        if (!localPageList.isEmpty()) {
+            page = localPageList.get(localPageList.size() - 1);
+        }
+        return page;
+    }
+
+    public synchronized Page getFirstPage() {
+        Page page = null;
+        if (!localPageList.isEmpty()) {
+            page = localPageList.get(0);
+        }
+        return page;
+    }
+
+    public synchronized FileReference getValidFileReference() throws InterruptedException {
+        while (fileRef == null)
+            wait();
+        return fileRef;
+    }
+
+    @Override
+    public JobId getJobId() {
+        return resultSetPartitionId.getJobId();
+    }
+
+    @Override
+    public Object getId() {
+        return resultSetPartitionId;
+    }
+
+    @Override
+    public long getMemoryOccupancy() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void toBytes(DataOutput out) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void fromBytes(DataInput in) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
new file mode 100644
index 0000000..5b8b333
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
@@ -0,0 +1,127 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.net;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.IChannelOpenListener;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
+
+public class DatasetNetworkManager implements IChannelConnectionFactory {
+    private static final Logger LOGGER = Logger.getLogger(DatasetNetworkManager.class.getName());
+
+    private static final int MAX_CONNECTION_ATTEMPTS = 5;
+
+    static final int INITIAL_MESSAGE_SIZE = 20;
+
+    private final IDatasetPartitionManager partitionManager;
+
+    private final MuxDemux md;
+
+    private NetworkAddress networkAddress;
+
+    public DatasetNetworkManager(InetAddress inetAddress, IDatasetPartitionManager partitionManager, int nThreads)
+            throws IOException {
+        this.partitionManager = partitionManager;
+        md = new MuxDemux(new InetSocketAddress(inetAddress, 0), new ChannelOpenListener(), nThreads,
+                MAX_CONNECTION_ATTEMPTS);
+    }
+
+    public void start() throws IOException {
+        md.start();
+        InetSocketAddress sockAddr = md.getLocalAddress();
+        networkAddress = new NetworkAddress(sockAddr.getAddress().getAddress(), sockAddr.getPort());
+    }
+
+    public NetworkAddress getNetworkAddress() {
+        return networkAddress;
+    }
+
+    public void stop() {
+
+    }
+
+    public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException {
+        MultiplexedConnection mConn = md.connect((InetSocketAddress) remoteAddress);
+        return mConn.openChannel();
+    }
+
+    private class ChannelOpenListener implements IChannelOpenListener {
+        @Override
+        public void channelOpened(ChannelControlBlock channel) {
+            channel.getReadInterface().setFullBufferAcceptor(new InitialBufferAcceptor(channel));
+            channel.getReadInterface().getEmptyBufferAcceptor().accept(ByteBuffer.allocate(INITIAL_MESSAGE_SIZE));
+        }
+    }
+
+    private class InitialBufferAcceptor implements ICloseableBufferAcceptor {
+        private final ChannelControlBlock ccb;
+
+        private NetworkOutputChannel noc;
+
+        public InitialBufferAcceptor(ChannelControlBlock ccb) {
+            this.ccb = ccb;
+        }
+
+        @Override
+        public void accept(ByteBuffer buffer) {
+            JobId jobId = new JobId(buffer.getLong());
+            int partition = buffer.getInt();
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Received initial dataset partition read request for JobId: " + jobId + " partition: "
+                        + partition + " on channel: " + ccb);
+            }
+            noc = new NetworkOutputChannel(ccb, 1);
+            try {
+                partitionManager.initializeDatasetPartitionReader(jobId, partition, noc);
+            } catch (HyracksException e) {
+                noc.abort();
+            }
+        }
+
+        @Override
+        public void close() {
+
+        }
+
+        @Override
+        public void error(int ecode) {
+            if (noc != null) {
+                noc.abort();
+            }
+        }
+    }
+
+    public MuxDemuxPerformanceCounters getPerformanceCounters() {
+        return md.getPerformanceCounters();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
index b805595..c8e4e94 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
@@ -27,6 +27,8 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
 import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
 import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
 import edu.uci.ics.hyracks.net.exceptions.NetException;
@@ -36,7 +38,7 @@
 import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
 import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
 
-public class NetworkManager {
+public class NetworkManager implements IChannelConnectionFactory {
     private static final Logger LOGGER = Logger.getLogger(NetworkManager.class.getName());
 
     private static final int MAX_CONNECTION_ATTEMPTS = 5;
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
index 16e31f7..ba6e6c3 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
@@ -21,7 +21,7 @@
 import edu.uci.ics.hyracks.api.channels.IInputChannel;
 import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.partitions.IPartition;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
@@ -82,7 +82,7 @@
     }
 
     @Override
-    public void open(IHyracksTaskContext ctx) throws HyracksDataException {
+    public void open(IHyracksCommonContext ctx) throws HyracksDataException {
         for (int i = 0; i < nBuffers; ++i) {
             emptyQueue.add(ctx.allocateFrame());
         }
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
index 45c091a..ea88a75 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
@@ -28,12 +28,12 @@
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.partitions.IPartition;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
 import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
 import edu.uci.ics.hyracks.control.common.job.PartitionState;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
-import edu.uci.ics.hyracks.control.nc.net.NetworkOutputChannel;
 import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
 
 public class PartitionManager {
@@ -98,7 +98,7 @@
         List<IPartition> pList = partitionMap.get(partitionId);
         if (pList != null && !pList.isEmpty()) {
             IPartition partition = pList.get(0);
-            writer.setTaskContext(partition.getTaskContext());
+            writer.setFrameSize(partition.getTaskContext().getFrameSize());
             partition.writeTo(writer);
             if (!partition.isReusable()) {
                 partitionMap.remove(partitionId);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ApplicationMessageWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ApplicationMessageWork.java
index deb1b75..a1499b8 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ApplicationMessageWork.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ApplicationMessageWork.java
@@ -19,35 +19,31 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.messages.IMessage;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
 import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
 
 /**
  * @author rico
- * 
  */
 public class ApplicationMessageWork extends AbstractWork {
-
     private static final Logger LOGGER = Logger.getLogger(ApplicationMessageWork.class.getName());
     private byte[] message;
     private String nodeId;
     private NodeControllerService ncs;
-    private String appName;
 
-    public ApplicationMessageWork(NodeControllerService ncs, byte[] message, String appName, String nodeId) {
+    public ApplicationMessageWork(NodeControllerService ncs, byte[] message, String nodeId) {
         this.ncs = ncs;
         this.nodeId = nodeId;
         this.message = message;
-        this.appName = appName;
     }
 
     @Override
     public void run() {
-
-        NCApplicationContext ctx = ncs.getApplications().get(appName);
+        NCApplicationContext ctx = ncs.getApplicationContext();
         try {
-            IMessage data = (IMessage) ctx.deserialize(message);
+            IMessage data = (IMessage) JavaSerializationUtils.deserialize(message);
             if (ctx.getMessageBroker() != null) {
                 ctx.getMessageBroker().receivedMessage(data, nodeId);
             } else {
@@ -64,5 +60,4 @@
     public String toString() {
         return "nodeID: " + nodeId;
     }
-
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
deleted file mode 100644
index 6eb1a95..0000000
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.nc.work;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.util.Map;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.http.HttpResponse;
-import org.apache.http.client.HttpClient;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.impl.client.DefaultHttpClient;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
-import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
-import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.nc.NodeControllerService;
-import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
-
-public class CreateApplicationWork extends AbstractWork {
-    private final NodeControllerService ncs;
-
-    private final String appName;
-
-    private final boolean deployHar;
-
-    private final byte[] serializedDistributedState;
-
-    public CreateApplicationWork(NodeControllerService ncs, String appName, boolean deployHar,
-            byte[] serializedDistributedState) {
-        this.ncs = ncs;
-        this.appName = appName;
-        this.deployHar = deployHar;
-        this.serializedDistributedState = serializedDistributedState;
-    }
-
-    @Override
-    public void run() {
-        try {
-            NCApplicationContext appCtx;
-            Map<String, NCApplicationContext> applications = ncs.getApplications();
-            if (applications.containsKey(appName)) {
-                throw new HyracksException("Duplicate application with name: " + appName + " being created.");
-            }
-            appCtx = new NCApplicationContext(ncs.getServerContext(), ncs.getRootContext(), appName, ncs.getId());
-            applications.put(appName, appCtx);
-            if (deployHar) {
-                NCConfig ncConfig = ncs.getConfiguration();
-                NodeParameters nodeParameters = ncs.getNodeParameters();
-                HttpClient hc = new DefaultHttpClient();
-                HttpGet get = new HttpGet("http://" + ncConfig.ccHost + ":"
-                        + nodeParameters.getClusterControllerInfo().getWebPort() + "/applications/" + appName);
-                HttpResponse response = hc.execute(get);
-                InputStream is = response.getEntity().getContent();
-                OutputStream os = appCtx.getHarOutputStream();
-                try {
-                    IOUtils.copyLarge(is, os);
-                } finally {
-                    os.close();
-                    is.close();
-                }
-            }
-            appCtx.initializeClassPath();
-            appCtx.setDistributedState((Serializable) appCtx.deserialize(serializedDistributedState));
-            appCtx.initialize();
-            ncs.getClusterController()
-                    .notifyApplicationStateChange(ncs.getId(), appName, ApplicationStatus.INITIALIZED);
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
deleted file mode 100644
index b104ce8..0000000
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.nc.work;
-
-import java.util.Map;
-
-import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
-import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.nc.NodeControllerService;
-import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
-
-public class DestroyApplicationWork extends AbstractWork {
-    private final NodeControllerService ncs;
-
-    private final String appName;
-
-    public DestroyApplicationWork(NodeControllerService ncs, String appName) {
-        this.ncs = ncs;
-        this.appName = appName;
-    }
-
-    @Override
-    public void run() {
-        try {
-            Map<String, NCApplicationContext> applications = ncs.getApplications();
-            ApplicationContext appCtx = applications.remove(appName);
-            if (appCtx != null) {
-                appCtx.deinitialize();
-            }
-            ncs.getClusterController().notifyApplicationStateChange(ncs.getId(), appName,
-                    ApplicationStatus.DEINITIALIZED);
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
index bb9669d..7ed9d11 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
@@ -22,10 +22,10 @@
 import edu.uci.ics.hyracks.api.comm.PartitionChannel;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.NetworkInputChannel;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 import edu.uci.ics.hyracks.control.nc.Joblet;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
-import edu.uci.ics.hyracks.control.nc.net.NetworkInputChannel;
 
 public class ReportPartitionAvailabilityWork extends AbstractWork {
     private final NodeControllerService ncs;
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
index 0c0fa3d..fffecc2 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
@@ -42,6 +42,7 @@
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
 import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 import edu.uci.ics.hyracks.control.nc.Joblet;
@@ -59,8 +60,6 @@
 
     private final NodeControllerService ncs;
 
-    private final String appName;
-
     private final JobId jobId;
 
     private final byte[] acgBytes;
@@ -71,11 +70,10 @@
 
     private final EnumSet<JobFlag> flags;
 
-    public StartTasksWork(NodeControllerService ncs, String appName, JobId jobId, byte[] acgBytes,
+    public StartTasksWork(NodeControllerService ncs, JobId jobId, byte[] acgBytes,
             List<TaskAttemptDescriptor> taskDescriptors,
             Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap, EnumSet<JobFlag> flags) {
         this.ncs = ncs;
-        this.appName = appName;
         this.jobId = jobId;
         this.acgBytes = acgBytes;
         this.taskDescriptors = taskDescriptors;
@@ -86,10 +84,9 @@
     @Override
     public void run() {
         try {
-            Map<String, NCApplicationContext> applications = ncs.getApplications();
-            NCApplicationContext appCtx = applications.get(appName);
+            NCApplicationContext appCtx = ncs.getApplicationContext();
             final Joblet joblet = getOrCreateLocalJoblet(jobId, appCtx, acgBytes == null ? null
-                    : (ActivityClusterGraph) appCtx.deserialize(acgBytes));
+                    : (ActivityClusterGraph) JavaSerializationUtils.deserialize(acgBytes));
             final ActivityClusterGraph acg = joblet.getActivityClusterGraph();
 
             IRecordDescriptorProvider rdp = new IRecordDescriptorProvider() {
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
index 8508287..c0ba163 100644
--- a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
@@ -21,6 +21,14 @@
 public class ByteArrayAccessibleOutputStream extends ByteArrayOutputStream {
     private static final Logger LOGGER = Logger.getLogger(ByteArrayAccessibleOutputStream.class.getName());
 
+    public ByteArrayAccessibleOutputStream() {
+        super();
+    }
+
+    public ByteArrayAccessibleOutputStream(int size) {
+        super(size);
+    }
+
     public byte[] getByteArray() {
         return buf;
     }
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
new file mode 100644
index 0000000..07f6ba2
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.common.comm.io;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+
+public class FrameOutputStream extends ByteArrayAccessibleOutputStream {
+    private static final Logger LOGGER = Logger.getLogger(FrameOutputStream.class.getName());
+
+    private final FrameTupleAppender frameTupleAppender;
+
+    public FrameOutputStream(int frameSize) {
+        super(frameSize);
+        this.frameTupleAppender = new FrameTupleAppender(frameSize);
+    }
+
+    public void reset(ByteBuffer buffer, boolean clear) {
+        frameTupleAppender.reset(buffer, clear);
+    }
+
+    public int getTupleCount() {
+        int tupleCount = frameTupleAppender.getTupleCount();
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("appendTuple(): tuple count: " + tupleCount);
+        }
+        return tupleCount;
+    }
+
+    public boolean appendTuple() {
+        if (LOGGER.isLoggable(Level.FINEST)) {
+            LOGGER.finest("appendTuple(): tuple size: " + count);
+        }
+        boolean appended = frameTupleAppender.append(buf, 0, count);
+        count = 0;
+        return appended;
+    }
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
new file mode 100644
index 0000000..915a436
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
@@ -0,0 +1,97 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.common.comm.io;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+public class ResultFrameTupleAccessor implements IFrameTupleAccessor {
+
+    private final int frameSize;
+    private ByteBuffer buffer;
+
+    public ResultFrameTupleAccessor(int frameSize) {
+        this.frameSize = frameSize;
+    }
+
+    @Override
+    public void reset(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    @Override
+    public int getTupleCount() {
+        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
+    }
+
+    @Override
+    public int getTupleStartOffset(int tupleIndex) {
+        return tupleIndex == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * tupleIndex);
+    }
+
+    @Override
+    public int getTupleEndOffset(int tupleIndex) {
+        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleIndex + 1));
+    }
+
+    @Override
+    public int getFieldStartOffset(int tupleIndex, int fIdx) {
+        return fIdx == 0 ? 0 : buffer.getInt(getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
+    }
+
+    @Override
+    public int getFieldEndOffset(int tupleIndex, int fIdx) {
+        return buffer.getInt(getTupleStartOffset(tupleIndex) + fIdx * 4);
+    }
+
+    @Override
+    public int getFieldLength(int tupleIndex, int fIdx) {
+        return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
+    }
+
+    @Override
+    public int getFieldSlotsLength() {
+        return getFieldCount() * 4;
+    }
+
+    public void prettyPrint() {
+        ByteBufferInputStream bbis = new ByteBufferInputStream();
+        DataInputStream dis = new DataInputStream(bbis);
+        int tc = getTupleCount();
+        System.err.println("TC: " + tc);
+        for (int i = 0; i < tc; ++i) {
+            System.err.print(i + ":(" + getTupleStartOffset(i) + ", " + getTupleEndOffset(i) + ")[");
+
+            bbis.setByteBuffer(buffer, getTupleStartOffset(i));
+            System.err.print(dis);
+
+            System.err.println("]");
+        }
+    }
+
+    @Override
+    public int getFieldCount() {
+        return 1;
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index f2b56fa..f86d9fb 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -256,8 +256,6 @@
 
                 outputAppender.reset(outputFrame, true);
 
-                writer.open();
-
                 if (tPointers == null) {
                     // Not sorted
                     for (int i = 0; i < tableSize; ++i) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
new file mode 100644
index 0000000..edca60a
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
@@ -0,0 +1,120 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.result;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+
+public class ResultWriterOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final ResultSetId rsId;
+
+    private final boolean ordered;
+
+    private final IResultSerializerFactory resultSerializerFactory;
+
+    public ResultWriterOperatorDescriptor(IOperatorDescriptorRegistry spec, ResultSetId rsId, boolean ordered,
+            IResultSerializerFactory resultSerializerFactory) throws IOException {
+        super(spec, 1, 0);
+        this.rsId = rsId;
+        this.ordered = ordered;
+        this.resultSerializerFactory = resultSerializerFactory;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+        final IDatasetPartitionManager dpm = ctx.getDatasetPartitionManager();
+
+        final ByteBuffer outputBuffer = ctx.allocateFrame();
+
+        final FrameOutputStream frameOutputStream = new FrameOutputStream(ctx.getFrameSize());
+        frameOutputStream.reset(outputBuffer, true);
+        PrintStream printStream = new PrintStream(frameOutputStream);
+
+        final RecordDescriptor outRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+        final IResultSerializer resultSerializer = resultSerializerFactory.createResultSerializer(outRecordDesc,
+                printStream);
+
+        final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), outRecordDesc);
+
+        return new AbstractUnaryInputSinkOperatorNodePushable() {
+            IFrameWriter datasetPartitionWriter;
+
+            @Override
+            public void open() throws HyracksDataException {
+                try {
+                    datasetPartitionWriter = dpm.createDatasetPartitionWriter(ctx, rsId, ordered, partition,
+                            nPartitions);
+                    datasetPartitionWriter.open();
+                    resultSerializer.init();
+                } catch (HyracksException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                frameTupleAccessor.reset(buffer);
+                for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+                    resultSerializer.appendTuple(frameTupleAccessor, tIndex);
+                    if (!frameOutputStream.appendTuple()) {
+                        datasetPartitionWriter.nextFrame(outputBuffer);
+                        frameOutputStream.reset(outputBuffer, true);
+
+                        /* TODO(madhusudancs): This works under the assumption that no single serialized record is
+                         * longer than the buffer size.
+                         */
+                        resultSerializer.appendTuple(frameTupleAccessor, tIndex);
+                        frameOutputStream.appendTuple();
+                    }
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                datasetPartitionWriter.fail();
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                if (frameOutputStream.getTupleCount() > 0) {
+                    datasetPartitionWriter.nextFrame(outputBuffer);
+                    frameOutputStream.reset(outputBuffer, true);
+                }
+                datasetPartitionWriter.close();
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/getip.sh b/hyracks/hyracks-dist/src/main/resources/bin/getip.sh
index e0cdf73..a691c0f 100755
--- a/hyracks/hyracks-dist/src/main/resources/bin/getip.sh
+++ b/hyracks/hyracks-dist/src/main/resources/bin/getip.sh
@@ -6,6 +6,10 @@
 then
         #Get IP Address
         IPADDR=`/sbin/ifconfig eth0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+    	if [ "$IPADDR" = "" ]
+        then
+		IPADDR=`/sbin/ifconfig em1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+        fi 
 	if [ "$IPADDR" = "" ]
         then
 		IPADDR=`/sbin/ifconfig lo | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh b/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh
index fe2551d..efb79ce 100755
--- a/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh
+++ b/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh
@@ -22,4 +22,4 @@
 
 #Launch hyracks cc script
 chmod -R 755 $HYRACKS_HOME
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 3 &> $CCLOGS_DIR/cc.log &
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh b/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh
index 6e0f90e..23a4c36 100755
--- a/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh
+++ b/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh
@@ -46,4 +46,4 @@
 cd $NCTMP_DIR
 
 #Launch hyracks nc
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -result-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
diff --git a/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh b/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh
index 03ce4e7..35c4794 100755
--- a/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh
+++ b/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh
@@ -5,6 +5,10 @@
 PID=`ps -ef|grep ${USER}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
 
 if [ "$PID" == "" ]; then
+  PID=`ps -ef|grep ${USER}|grep java|grep 'hyracks'|awk '{print $2}'`
+fi
+
+if [ "$PID" == "" ]; then
   USERID=`id | sed 's/^uid=//;s/(.*$//'`
   PID=`ps -ef|grep ${USERID}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
 fi
diff --git a/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml b/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml
deleted file mode 100644
index 792cedd..0000000
--- a/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml
+++ /dev/null
@@ -1,87 +0,0 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <groupId>edu.uci.ics.hyracks.examples.btree</groupId>
-  <artifactId>btreeapp</artifactId>
-  <name>btreeapp</name>
-
-  <parent>
-    <groupId>edu.uci.ics.hyracks.examples</groupId>
-    <artifactId>btree-example</artifactId>
-    <version>0.2.3-SNAPSHOT</version>
-  </parent>
-
-  <build>
-    <pluginManagement>
-      <plugins>
-        <plugin>
-          <groupId>org.eclipse.m2e</groupId>
-          <artifactId>lifecycle-mapping</artifactId>
-          <version>1.0.0</version>
-          <configuration>
-            <lifecycleMappingMetadata>
-              <pluginExecutions>
-                <pluginExecution>
-                  <pluginExecutionFilter>
-                    <groupId>org.apache.maven.plugins</groupId>
-                    <artifactId>maven-dependency-plugin</artifactId>
-                    <versionRange>[1.0.0,)</versionRange>
-                    <goals>
-                      <goal>copy-dependencies</goal>
-                    </goals>
-                  </pluginExecutionFilter>
-                  <action>
-                    <ignore />
-                  </action>
-                </pluginExecution>
-              </pluginExecutions>
-            </lifecycleMappingMetadata>
-          </configuration>
-        </plugin>
-      </plugins>
-	</pluginManagement>
-  
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>copy-dependencies</id>
-            <phase>package</phase>
-            <goals>
-              <goal>copy-dependencies</goal>
-            </goals>
-            <configuration>
-              <outputDirectory>target/application/lib</outputDirectory>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <artifactId>maven-assembly-plugin</artifactId>
-        <version>2.2-beta-5</version>
-        <executions>
-          <execution>
-            <configuration>
-              <descriptors>
-                <descriptor>src/main/assembly/app-assembly.xml</descriptor>
-              </descriptors>
-            </configuration>
-            <phase>package</phase>
-            <goals>
-              <goal>attached</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-  <dependencies>
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks.examples.btree</groupId>
-  		<artifactId>btreehelper</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<scope>compile</scope>
-  	</dependency>
-  </dependencies>
-</project>
diff --git a/hyracks/hyracks-examples/btree-example/btreeapp/src/main/assembly/app-assembly.xml b/hyracks/hyracks-examples/btree-example/btreeapp/src/main/assembly/app-assembly.xml
deleted file mode 100644
index 43ace6c..0000000
--- a/hyracks/hyracks-examples/btree-example/btreeapp/src/main/assembly/app-assembly.xml
+++ /dev/null
@@ -1,13 +0,0 @@
-<assembly>
-  <id>app-assembly</id>
-  <formats>
-    <format>zip</format>
-  </formats>
-  <includeBaseDirectory>false</includeBaseDirectory>
-  <fileSets>
-    <fileSet>
-      <directory>target/application/lib</directory>
-      <outputDirectory>lib</outputDirectory>
-    </fileSet>
-  </fileSets>
-</assembly>
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
index 86c11d3..a939a04 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
@@ -61,9 +61,6 @@
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
         public int port = 1098;
 
-        @Option(name = "-app", usage = "Hyracks Application name", required = true)
-        public String app;
-
         @Option(name = "-target-ncs", usage = "Comma separated list of node-controller names to use", required = true)
         public String ncs;
 
@@ -87,7 +84,7 @@
         JobSpecification job = createJob(options);
 
         long start = System.currentTimeMillis();
-        JobId jobId = hcc.startJob(options.app, job);
+        JobId jobId = hcc.startJob(job);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
         System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index 105fe9b..616fdae 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -59,9 +59,6 @@
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
         public int port = 1098;
 
-        @Option(name = "-app", usage = "Hyracks Application name", required = true)
-        public String app;
-
         @Option(name = "-target-ncs", usage = "Comma separated list of node-controller names to use", required = true)
         public String ncs;
 
@@ -85,7 +82,7 @@
         JobSpecification job = createJob(options);
 
         long start = System.currentTimeMillis();
-        JobId jobId = hcc.startJob(options.app, job);
+        JobId jobId = hcc.startJob(job);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
         System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index afc3487..3493947 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -58,9 +58,6 @@
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
         public int port = 1098;
 
-        @Option(name = "-app", usage = "Hyracks Application name", required = true)
-        public String app;
-
         @Option(name = "-target-ncs", usage = "Comma separated list of node-controller names to use", required = true)
         public String ncs;
 
@@ -78,7 +75,7 @@
         JobSpecification job = createJob(options);
 
         long start = System.currentTimeMillis();
-        JobId jobId = hcc.startJob(options.app, job);
+        JobId jobId = hcc.startJob(job);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
         System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index 0c6ab8e..075ea20 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -54,9 +54,6 @@
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
         public int port = 1098;
 
-        @Option(name = "-app", usage = "Hyracks Application name", required = true)
-        public String app;
-
         @Option(name = "-target-ncs", usage = "Comma separated list of node-controller names to use", required = true)
         public String ncs;
 
@@ -80,7 +77,7 @@
         JobSpecification job = createJob(options);
 
         long start = System.currentTimeMillis();
-        JobId jobId = hcc.startJob(options.app, job);
+        JobId jobId = hcc.startJob(job);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
         System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index e2c0be9..a6ffc13 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -58,9 +58,6 @@
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
         public int port = 1098;
 
-        @Option(name = "-app", usage = "Hyracks Application name", required = true)
-        public String app;
-
         @Option(name = "-target-ncs", usage = "Comma separated list of node-controller names to use", required = true)
         public String ncs;
 
@@ -81,7 +78,7 @@
         JobSpecification job = createJob(options);
 
         long start = System.currentTimeMillis();
-        JobId jobId = hcc.startJob(options.app, job);
+        JobId jobId = hcc.startJob(job);
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
         System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/NCApplicationEntryPoint.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/NCApplicationEntryPoint.java
new file mode 100644
index 0000000..a1dacd8
--- /dev/null
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/NCApplicationEntryPoint.java
@@ -0,0 +1,22 @@
+package edu.uci.ics.hyracks.examples.btree.helper;
+
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.application.INCApplicationEntryPoint;
+
+public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
+    @Override
+    public void start(INCApplicationContext ncAppCtx, String[] args) throws Exception {
+        RuntimeContext rCtx = new RuntimeContext(ncAppCtx);
+        ncAppCtx.setApplicationObject(rCtx);
+    }
+
+    @Override
+    public void notifyStartupComplete() throws Exception {
+
+    }
+
+    @Override
+    public void stop() throws Exception {
+
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/NCBootstrap.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/NCBootstrap.java
deleted file mode 100644
index ea55e7a..0000000
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/NCBootstrap.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.hyracks.examples.btree.helper;
-
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.application.INCApplicationContext;
-import edu.uci.ics.hyracks.api.application.INCBootstrap;
-
-public class NCBootstrap implements INCBootstrap {
-    private static final Logger LOGGER = Logger.getLogger(NCBootstrap.class.getName());
-
-    private INCApplicationContext appCtx;
-
-    @Override
-    public void start() throws Exception {
-        LOGGER.info("Starting NC Bootstrap");
-        RuntimeContext rCtx = new RuntimeContext(appCtx);
-        appCtx.setApplicationObject(rCtx);
-        LOGGER.info("Initialized RuntimeContext: " + rCtx);
-    }
-
-    @Override
-    public void stop() throws Exception {
-        LOGGER.info("Stopping Asterix NC Bootstrap");
-        RuntimeContext rCtx = (RuntimeContext) appCtx.getApplicationObject();
-        rCtx.close();
-    }
-
-    @Override
-    public void setApplicationContext(INCApplicationContext appCtx) {
-        this.appCtx = appCtx;
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/resources/hyracks-deployment.properties b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/resources/hyracks-deployment.properties
deleted file mode 100644
index ab0ecb3..0000000
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/resources/hyracks-deployment.properties
+++ /dev/null
@@ -1 +0,0 @@
-nc.bootstrap.class=edu.uci.ics.hyracks.examples.btree.helper.NCBootstrap
\ No newline at end of file
diff --git a/hyracks/hyracks-cli/pom.xml b/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
similarity index 60%
rename from hyracks/hyracks-cli/pom.xml
rename to hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
index 280478d..ec6747d 100644
--- a/hyracks/hyracks-cli/pom.xml
+++ b/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
@@ -1,43 +1,19 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
   <modelVersion>4.0.0</modelVersion>
-  <artifactId>hyracks-cli</artifactId>
-  <name>hyracks-cli</name>
+  <groupId>edu.uci.ics.hyracks.examples.btree</groupId>
+  <artifactId>btreeserver</artifactId>
+  <version>0.2.3-SNAPSHOT</version>
+  <name>btreeserver</name>
 
   <parent>
-    <groupId>edu.uci.ics.hyracks</groupId>
-    <artifactId>hyracks</artifactId>
+    <groupId>edu.uci.ics.hyracks.examples</groupId>
+    <artifactId>btree-example</artifactId>
     <version>0.2.3-SNAPSHOT</version>
   </parent>
 
   <build>
     <plugins>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <version>2.0.2</version>
-        <configuration>
-          <source>1.7</source>
-          <target>1.7</target>
-          <fork>true</fork>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>javacc-maven-plugin</artifactId>
-        <version>2.6</version>
-        <executions>
-          <execution>
-            <id>javacc</id>
-            <goals>
-              <goal>javacc</goal>
-            </goals>
-            <configuration>
-              <isStatic>false</isStatic>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
         <version>1.3</version>
@@ -46,8 +22,12 @@
             <configuration>
               <programs>
                 <program>
-                  <mainClass>edu.uci.ics.hyracks.cli.Main</mainClass>
-                  <name>hyrackscli</name>
+                  <mainClass>edu.uci.ics.hyracks.control.cc.CCDriver</mainClass>
+                  <name>hyrackscc</name>
+                </program>
+                <program>
+                  <mainClass>edu.uci.ics.hyracks.control.nc.NCDriver</mainClass>
+                  <name>hyracksnc</name>
                 </program>
               </programs>
               <repositoryLayout>flat</repositoryLayout>
@@ -81,16 +61,23 @@
   </build>
   <dependencies>
   	<dependency>
-  		<groupId>jline</groupId>
-  		<artifactId>jline</artifactId>
-  		<version>0.9.94</version>
+  		<groupId>edu.uci.ics.hyracks.examples.btree</groupId>
+  		<artifactId>btreehelper</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-cc</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-api</artifactId>
+  		<artifactId>hyracks-control-nc</artifactId>
   		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   </dependencies>
diff --git a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml b/hyracks/hyracks-examples/btree-example/btreeserver/src/main/assembly/binary-assembly.xml
similarity index 81%
rename from hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
rename to hyracks/hyracks-examples/btree-example/btreeserver/src/main/assembly/binary-assembly.xml
index 0500499..cd598d9 100644
--- a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
+++ b/hyracks/hyracks-examples/btree-example/btreeserver/src/main/assembly/binary-assembly.xml
@@ -15,5 +15,9 @@
       <directory>target/appassembler/lib</directory>
       <outputDirectory>lib</outputDirectory>
     </fileSet>
+    <fileSet>
+      <directory>docs</directory>
+      <outputDirectory>docs</outputDirectory>
+    </fileSet>
   </fileSets>
 </assembly>
diff --git a/hyracks/hyracks-examples/btree-example/pom.xml b/hyracks/hyracks-examples/btree-example/pom.xml
index 9ef5df5..6614a6e 100644
--- a/hyracks/hyracks-examples/btree-example/pom.xml
+++ b/hyracks/hyracks-examples/btree-example/pom.xml
@@ -14,6 +14,6 @@
   <modules>
     <module>btreehelper</module>
     <module>btreeclient</module>
-    <module>btreeapp</module>
+    <module>btreeserver</module>
   </modules>
 </project>
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/src/main/assembly/app-assembly.xml b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/src/main/assembly/app-assembly.xml
deleted file mode 100644
index 43ace6c..0000000
--- a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/src/main/assembly/app-assembly.xml
+++ /dev/null
@@ -1,13 +0,0 @@
-<assembly>
-  <id>app-assembly</id>
-  <formats>
-    <format>zip</format>
-  </formats>
-  <includeBaseDirectory>false</includeBaseDirectory>
-  <fileSets>
-    <fileSet>
-      <directory>target/application/lib</directory>
-      <outputDirectory>lib</outputDirectory>
-    </fileSet>
-  </fileSets>
-</assembly>
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/conf/local_cluster.conf b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/conf/local_cluster.conf
similarity index 100%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/conf/local_cluster.conf
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/conf/local_cluster.conf
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/data/file1.txt b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/data/file1.txt
similarity index 100%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/data/file1.txt
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/data/file1.txt
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/data/file2.txt b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/data/file2.txt
similarity index 100%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/data/file2.txt
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/data/file2.txt
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/job/wordcount/wordcount.job b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/job/wordcount/wordcount.job
similarity index 100%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/job/wordcount/wordcount.job
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/job/wordcount/wordcount.job
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/pom.xml
similarity index 60%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/pom.xml
index 4cf9745..b538db5 100644
--- a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
+++ b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/pom.xml
@@ -1,8 +1,9 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.compat</groupId>
-  <artifactId>hadoopcompatapp</artifactId>
-  <name>hadoopcompatapp</name>
+  <artifactId>hadoopcompatserver</artifactId>
+  <version>0.2.3-SNAPSHOT</version>
+  <name>hadoopcompatserver</name>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
@@ -11,49 +12,31 @@
   </parent>
 
   <build>
-    <pluginManagement>
-      <plugins>
-        <plugin>
-          <groupId>org.eclipse.m2e</groupId>
-          <artifactId>lifecycle-mapping</artifactId>
-          <version>1.0.0</version>
-          <configuration>
-            <lifecycleMappingMetadata>
-              <pluginExecutions>
-                <pluginExecution>
-                  <pluginExecutionFilter>
-                    <groupId>org.apache.maven.plugins</groupId>
-                    <artifactId>maven-dependency-plugin</artifactId>
-                    <versionRange>[1.0.0,)</versionRange>
-                    <goals>
-                      <goal>copy-dependencies</goal>
-                    </goals>
-                  </pluginExecutionFilter>
-                  <action>
-                    <ignore />
-                  </action>
-                </pluginExecution>
-              </pluginExecutions>
-            </lifecycleMappingMetadata>
-          </configuration>
-        </plugin>
-      </plugins>
-	</pluginManagement>
-  
     <plugins>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
-            <id>copy-dependencies</id>
+            <configuration>
+              <programs>
+                <program>
+                  <mainClass>edu.uci.ics.hyracks.control.cc.CCDriver</mainClass>
+                  <name>hyrackscc</name>
+                </program>
+                <program>
+                  <mainClass>edu.uci.ics.hyracks.control.nc.NCDriver</mainClass>
+                  <name>hyracksnc</name>
+                </program>
+              </programs>
+              <repositoryLayout>flat</repositoryLayout>
+              <repositoryName>lib</repositoryName>
+            </configuration>
             <phase>package</phase>
             <goals>
-              <goal>copy-dependencies</goal>
+              <goal>assemble</goal>
             </goals>
-            <configuration>
-              <outputDirectory>target/application/lib</outputDirectory>
-            </configuration>
           </execution>
         </executions>
       </plugin>
@@ -64,7 +47,7 @@
           <execution>
             <configuration>
               <descriptors>
-                <descriptor>src/main/assembly/app-assembly.xml</descriptor>
+                <descriptor>src/main/assembly/binary-assembly.xml</descriptor>
               </descriptors>
             </configuration>
             <phase>package</phase>
@@ -79,8 +62,7 @@
       	<artifactId>hyracks-virtualcluster-maven-plugin</artifactId>
       	<version>0.2.3-SNAPSHOT</version>
         <configuration>
-          <hyracksServerHome>${basedir}/../../../hyracks-server/target/hyracks-server-${project.version}-binary-assembly</hyracksServerHome>
-          <hyracksCLIHome>${basedir}/../../../hyracks-cli/target/hyracks-cli-${project.version}-binary-assembly</hyracksCLIHome>
+          <hyracksServerHome>${basedir}/target/hadoopcompatserver-${project.version}-binary-assembly</hyracksServerHome>
           <jvmOptions>${jvm.extraargs}</jvmOptions>
         </configuration>
         <executions>
@@ -90,9 +72,6 @@
             <goals>
               <goal>start-cc</goal>
             </goals>
-            <configuration>
-	      <workingDir>${project.build.directory}</workingDir>
-            </configuration>
           </execution>
           <execution>
             <id>hyracks-nc1-start</id>
@@ -104,7 +83,6 @@
               <nodeId>NC1</nodeId>
               <dataIpAddress>127.0.0.1</dataIpAddress>
               <ccHost>localhost</ccHost>
-	      <workingDir>${project.build.directory}</workingDir>
             </configuration>
           </execution>
           <execution>
@@ -117,38 +95,24 @@
               <nodeId>NC2</nodeId>
               <dataIpAddress>127.0.0.1</dataIpAddress>
               <ccHost>localhost</ccHost>
-	      <workingDir>${project.build.directory}</workingDir>
             </configuration>
           </execution>
           <execution>
-            <id>deploy-app</id>
-            <phase>pre-integration-test</phase>
+            <id>stop-services</id>
+            <phase>post-integration-test</phase>
             <goals>
-              <goal>deploy-app</goal>
+              <goal>stop-services</goal>
             </goals>
-            <configuration>
-              <ccHost>localhost</ccHost>
-              <appName>compat</appName>
-              <harFile>${project.build.directory}/hadoopcompatapp-${project.version}-app-assembly.zip</harFile>
-            </configuration>
           </execution>
-	     <execution>
-	       <id>stop-services</id>
-	       <phase>post-integration-test</phase>
-	       <goals>
-	         <goal>stop-services</goal>
-	       </goals>
-             </execution>
-          </executions>
+        </executions>
       </plugin>
       <plugin>
       	<groupId>org.apache.maven.plugins</groupId>
       	<artifactId>maven-compiler-plugin</artifactId>
       	<version>2.0.2</version>
         <configuration>
-          <source>1.7</source>
-          <target>1.7</target>
-          <fork>true</fork>
+          <source>1.6</source>
+          <target>1.6</target>
         </configuration>
       </plugin>
       <plugin>
@@ -174,6 +138,20 @@
         <version>0.2.3-SNAPSHOT</version>
         <scope>compile</scope>
      </dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-cc</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-nc</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
      <dependency>
         <groupId>edu.uci.ics.hyracks.examples.compat</groupId>
   	    <artifactId>hadoopcompatclient</artifactId>
diff --git a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/src/main/assembly/binary-assembly.xml
similarity index 81%
copy from hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
copy to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/src/main/assembly/binary-assembly.xml
index 0500499..cd598d9 100644
--- a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
+++ b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/src/main/assembly/binary-assembly.xml
@@ -15,5 +15,9 @@
       <directory>target/appassembler/lib</directory>
       <outputDirectory>lib</outputDirectory>
     </fileSet>
+    <fileSet>
+      <directory>docs</directory>
+      <outputDirectory>docs</outputDirectory>
+    </fileSet>
   </fileSets>
 </assembly>
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/src/test/java/edu/uci/ics/hyracks/examples/compat/test/WordCountCompatibilityIT.java b/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/src/test/java/edu/uci/ics/hyracks/examples/compat/test/WordCountCompatibilityIT.java
similarity index 100%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/src/test/java/edu/uci/ics/hyracks/examples/compat/test/WordCountCompatibilityIT.java
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/src/test/java/edu/uci/ics/hyracks/examples/compat/test/WordCountCompatibilityIT.java
diff --git a/hyracks/hyracks-examples/hadoop-compat-example/pom.xml b/hyracks/hyracks-examples/hadoop-compat-example/pom.xml
index 16787ca..103d762 100644
--- a/hyracks/hyracks-examples/hadoop-compat-example/pom.xml
+++ b/hyracks/hyracks-examples/hadoop-compat-example/pom.xml
@@ -14,7 +14,7 @@
   <modules>
     <module>hadoopcompathelper</module>
     <module>hadoopcompatclient</module>
-    <module>hadoopcompatapp</module>
+    <module>hadoopcompatserver</module>
   </modules>
 
   <dependencies>
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
index 25c7655..30d47ea 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -9,7 +9,6 @@
 		<artifactId>hyracks-examples</artifactId>
 		<version>0.2.3-SNAPSHOT</version>
 	</parent>
-
 	<build>
 		<plugins>
 			<plugin>
@@ -99,5 +98,12 @@
 			<artifactId>hyracks-data-std</artifactId>
 			<version>0.2.3-SNAPSHOT</version>
 		</dependency>
+  	    <dependency>
+            <groupId>edu.uci.ics.hyracks</groupId>
+            <artifactId>hyracks-client</artifactId>
+            <version>0.2.3-SNAPSHOT</version>
+            <type>jar</type>
+            <scope>compile</scope>
+        </dependency>
 	</dependencies>
 </project>
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index 0e0e899..e74e54c 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -14,29 +14,39 @@
  */
 package edu.uci.ics.hyracks.tests.integration;
 
+import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileReader;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.commons.io.FileUtils;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
 
 import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.client.dataset.HyracksDataset;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
 public abstract class AbstractIntegrationTest {
     private static final Logger LOGGER = Logger.getLogger(AbstractIntegrationTest.class.getName());
@@ -83,6 +93,7 @@
         ncConfig1.ccPort = 39001;
         ncConfig1.clusterNetIPAddress = "127.0.0.1";
         ncConfig1.dataIPAddress = "127.0.0.1";
+        ncConfig1.datasetIPAddress = "127.0.0.1";
         ncConfig1.nodeId = NC1_ID;
         nc1 = new NodeControllerService(ncConfig1);
         nc1.start();
@@ -92,12 +103,12 @@
         ncConfig2.ccPort = 39001;
         ncConfig2.clusterNetIPAddress = "127.0.0.1";
         ncConfig2.dataIPAddress = "127.0.0.1";
+        ncConfig2.datasetIPAddress = "127.0.0.1";
         ncConfig2.nodeId = NC2_ID;
         nc2 = new NodeControllerService(ncConfig2);
         nc2.start();
 
         hcc = new HyracksConnection(ccConfig.clientNetIpAddress, ccConfig.clientNetPort);
-        hcc.createApplication("test", null);
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting CC in " + ccRoot.getAbsolutePath());
         }
@@ -110,33 +121,80 @@
         cc.stop();
     }
 
-    protected void runTest(JobSpecification spec) throws Exception {
+    protected JobId executeTest(JobSpecification spec) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(spec.toJSON().toString(2));
         }
-        JobId jobId = hcc.startJob("test", spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+        JobId jobId = hcc.startJob(spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(jobId.toString());
         }
-        hcc.waitForCompletion(jobId);
-        dumpOutputFiles();
+        return jobId;
     }
 
-    private void dumpOutputFiles() {
-        if (LOGGER.isLoggable(Level.INFO)) {
-            for (File f : outputFiles) {
-                if (f.exists() && f.isFile()) {
-                    try {
-                        LOGGER.info("Reading file: " + f.getAbsolutePath() + " in test: " + getClass().getName());
-                        String data = FileUtils.readFileToString(f);
-                        LOGGER.info(data);
-                    } catch (IOException e) {
-                        LOGGER.info("Error reading file: " + f.getAbsolutePath());
-                        LOGGER.info(e.getMessage());
-                    }
+    protected void runTest(JobSpecification spec) throws Exception {
+        JobId jobId = executeTest(spec);
+        hcc.waitForCompletion(jobId);
+    }
+
+    protected List<String> readResults(JobSpecification spec, JobId jobId, ResultSetId resultSetId) throws Exception {
+        int nReaders = 1;
+        ByteBuffer resultBuffer = ByteBuffer.allocate(spec.getFrameSize());
+        resultBuffer.clear();
+
+        IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor(spec.getFrameSize());
+
+        IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
+        IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, resultSetId);
+
+        List<String> resultRecords = new ArrayList<String>();
+        ByteBufferInputStream bbis = new ByteBufferInputStream();
+
+        int readSize = reader.read(resultBuffer);
+
+        while (readSize > 0) {
+
+            try {
+                frameTupleAccessor.reset(resultBuffer);
+                for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+                    int start = frameTupleAccessor.getTupleStartOffset(tIndex);
+                    int length = frameTupleAccessor.getTupleEndOffset(tIndex) - start;
+                    bbis.setByteBuffer(resultBuffer, start);
+                    byte[] recordBytes = new byte[length];
+                    bbis.read(recordBytes, 0, length);
+                    resultRecords.add(new String(recordBytes, 0, length));
                 }
+            } finally {
+                bbis.close();
             }
+
+            resultBuffer.clear();
+            readSize = reader.read(resultBuffer);
         }
+        return resultRecords;
+    }
+
+    protected boolean runTestAndCompareResults(JobSpecification spec, String[] expectedFileNames) throws Exception {
+        JobId jobId = executeTest(spec);
+
+        List<String> results;
+        for (int i = 0; i < expectedFileNames.length; i++) {
+            results = readResults(spec, jobId, spec.getResultSetIds().get(i));
+            BufferedReader expectedFile = new BufferedReader(new FileReader(expectedFileNames[i]));
+
+            String expectedLine, actualLine;
+            int j = 0;
+            while ((expectedLine = expectedFile.readLine()) != null) {
+                actualLine = results.get(j).trim();
+                Assert.assertEquals(expectedLine, actualLine);
+                j++;
+            }
+            Assert.assertEquals(j, results.size());
+            expectedFile.close();
+        }
+
+        hcc.waitForCompletion(jobId);
+        return true;
     }
 
     protected File createTempFile() throws IOException {
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
index e0b8c73..d97b7db 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
@@ -16,6 +16,7 @@
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
@@ -23,6 +24,7 @@
 import java.util.logging.Logger;
 
 import org.apache.commons.io.FileUtils;
+import org.json.JSONArray;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -30,13 +32,20 @@
 
 import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.client.dataset.HyracksDataset;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
 public abstract class AbstractMultiNCIntegrationTest {
 
@@ -84,13 +93,13 @@
             ncConfig.ccPort = 39001;
             ncConfig.clusterNetIPAddress = "127.0.0.1";
             ncConfig.dataIPAddress = "127.0.0.1";
+            ncConfig.datasetIPAddress = "127.0.0.1";
             ncConfig.nodeId = ASTERIX_IDS[i];
             asterixNCs[i] = new NodeControllerService(ncConfig);
             asterixNCs[i].start();
         }
 
         hcc = new HyracksConnection(ccConfig.clientNetIpAddress, ccConfig.clientNetPort);
-        hcc.createApplication("test", null);
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting CC in " + ccRoot.getAbsolutePath());
         }
@@ -108,10 +117,50 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(spec.toJSON().toString(2));
         }
-        JobId jobId = hcc.startJob("test", spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+        JobId jobId = hcc.startJob(spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(jobId.toString());
         }
+
+        int nReaders = 1;
+
+        ByteBuffer resultBuffer = ByteBuffer.allocate(spec.getFrameSize());
+        resultBuffer.clear();
+
+        IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor(spec.getFrameSize());
+
+        IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
+        IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, spec.getResultSetIds().get(0));
+
+        JSONArray resultRecords = new JSONArray();
+        ByteBufferInputStream bbis = new ByteBufferInputStream();
+
+        int readSize = reader.read(resultBuffer);
+
+        while (readSize > 0) {
+
+            try {
+                frameTupleAccessor.reset(resultBuffer);
+                for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+                    int start = frameTupleAccessor.getTupleStartOffset(tIndex);
+                    int length = frameTupleAccessor.getTupleEndOffset(tIndex) - start;
+                    bbis.setByteBuffer(resultBuffer, start);
+                    byte[] recordBytes = new byte[length];
+                    bbis.read(recordBytes, 0, length);
+                    resultRecords.put(new String(recordBytes, 0, length));
+                }
+            } finally {
+                try {
+                    bbis.close();
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            resultBuffer.clear();
+            readSize = reader.read(resultBuffer);
+        }
+
         hcc.waitForCompletion(jobId);
         dumpOutputFiles();
     }
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
index 93e1e9b..06751e3 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
@@ -25,8 +25,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
@@ -49,7 +49,6 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableTableFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.AvgFieldGroupAggregatorFactory;
@@ -62,60 +61,42 @@
 import edu.uci.ics.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 /**
  *
  */
 public class AggregationTest extends AbstractIntegrationTest {
 
-    final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
-                    "data/tpch0.001/lineitem.tbl"))) });
+    final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC2_ID,
+            new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
 
-    final RecordDescriptor desc = new RecordDescriptor(
-            new ISerializerDeserializer[] {
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    IntegerSerializerDeserializer.INSTANCE,
-                    IntegerSerializerDeserializer.INSTANCE,
-                    IntegerSerializerDeserializer.INSTANCE,
-                    IntegerSerializerDeserializer.INSTANCE,
-                    FloatSerializerDeserializer.INSTANCE,
-                    FloatSerializerDeserializer.INSTANCE,
-                    FloatSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE,
-                    UTF8StringSerializerDeserializer.INSTANCE });
+    final RecordDescriptor desc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+            FloatSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
-    final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(
-            new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                    IntegerParserFactory.INSTANCE,
-                    IntegerParserFactory.INSTANCE,
-                    IntegerParserFactory.INSTANCE,
-                    IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
-                    FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE,
-                    UTF8StringParserFactory.INSTANCE, }, '|');
+    final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
+            UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+            IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
+            FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, }, '|');
 
-    private AbstractSingleActivityOperatorDescriptor getPrinter(
-            IOperatorDescriptorRegistry spec, String prefix) throws IOException {
+    private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification spec, String prefix)
+            throws IOException {
 
-        AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(
-                spec, new ConstantFileSplitProvider(new FileSplit[] {
-                        new FileSplit(NC1_ID, createTempFile()
-                                .getAbsolutePath()),
-                        new FileSplit(NC2_ID, createTempFile()
-                                .getAbsolutePath()) }), "\t");
+        ResultSetId rsId = new ResultSetId(1);
+        AbstractSingleActivityOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
 
         return printer;
     }
@@ -124,54 +105,38 @@
     public void singleKeySumInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new IntSumFieldAggregatorFactory(3, true),
-                                new FloatSumFieldAggregatorFactory(5, true) }),
-                outputRec, tableSize);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
+                        new FloatSumFieldAggregatorFactory(5, true) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -184,49 +149,34 @@
     public void singleKeySumPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE});
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new IntSumFieldAggregatorFactory(3, true),
-                                new FloatSumFieldAggregatorFactory(5, true)}),
-                outputRec);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
+                        new FloatSumFieldAggregatorFactory(5, true) }), outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -239,64 +189,43 @@
     public void singleKeySumExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE});
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
                                 new IntSumFieldAggregatorFactory(3, false),
-                                new FloatSumFieldAggregatorFactory(5, false)}),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
+                                new FloatSumFieldAggregatorFactory(5, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
                                 new IntSumFieldAggregatorFactory(2, false),
-                                new FloatSumFieldAggregatorFactory(3, false)}),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }),
-                        tableSize), true);
+                                new FloatSumFieldAggregatorFactory(3, false) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }), tableSize), true);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeySumExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -309,54 +238,38 @@
     public void singleKeyAvgInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new CountFieldAggregatorFactory(true),
-                                new AvgFieldGroupAggregatorFactory(1, true) }),
-                outputRec, tableSize);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -369,49 +282,34 @@
     public void singleKeyAvgPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new CountFieldAggregatorFactory(true),
-                                new AvgFieldGroupAggregatorFactory(1, true) }),
-                outputRec);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -424,64 +322,43 @@
     public void singleKeyAvgExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new CountFieldAggregatorFactory(false),
-                                new AvgFieldGroupAggregatorFactory(1, false) }),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
+                        new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
                                 new IntSumFieldAggregatorFactory(2, false),
-                                new AvgFieldMergeAggregatorFactory(3, false) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }),
-                        tableSize), true);
+                                new AvgFieldMergeAggregatorFactory(3, false) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }), tableSize), true);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -494,52 +371,38 @@
     public void singleKeyMinMaxStringInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, false) }), outputRec, tableSize);
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true),
+                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -552,47 +415,34 @@
     public void singleKeyMinMaxStringPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, false) }), outputRec);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true),
+                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -605,63 +455,42 @@
     public void singleKeyMinMaxStringExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                        .of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, true) }),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new MinMaxStringFieldAggregatorFactory(2, true,
-                                        true) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }),
-                        tableSize), true);
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new MinMaxStringFieldAggregatorFactory(15, true, true) }),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false),
+                        new MinMaxStringFieldAggregatorFactory(2, true, true) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }), tableSize), true);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec,
-                new FieldHashPartitionComputerFactory(
-                        keyFields,
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "singleKeyAvgExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -674,58 +503,39 @@
     public void multiKeySumInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec, keyFields, new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new IntSumFieldAggregatorFactory(3, true) }),
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
                 outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -738,51 +548,35 @@
     public void multiKeySumPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec, keyFields,
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new IntSumFieldAggregatorFactory(3, true) }),
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
                 outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -795,69 +589,43 @@
     public void multiKeySumExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new IntSumFieldAggregatorFactory(3, false) }),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(2, false),
-                                new IntSumFieldAggregatorFactory(3, false) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] {
-                                        PointableBinaryHashFunctionFactory
-                                                .of(UTF8StringPointable.FACTORY),
-                                        PointableBinaryHashFunctionFactory
-                                                .of(UTF8StringPointable.FACTORY) }),
-                        tableSize), true);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new IntSumFieldAggregatorFactory(3, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
+                                new IntSumFieldAggregatorFactory(3, false) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeySumExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -870,60 +638,40 @@
     public void multiKeyAvgInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec, keyFields, new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new CountFieldAggregatorFactory(true),
-                                new AvgFieldGroupAggregatorFactory(1, true) }),
-                outputRec, tableSize);
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -936,53 +684,36 @@
     public void multiKeyAvgPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec, keyFields,
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new CountFieldAggregatorFactory(true),
-                                new AvgFieldGroupAggregatorFactory(1, true) }),
-                outputRec);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyAvgInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -995,72 +726,46 @@
     public void multiKeyAvgExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new CountFieldAggregatorFactory(false),
-                                new AvgFieldGroupAggregatorFactory(1, false) }),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(2, false),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
+                        new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
                                 new IntSumFieldAggregatorFactory(3, false),
-                                new AvgFieldMergeAggregatorFactory(4, false) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] {
-                                        PointableBinaryHashFunctionFactory
-                                                .of(UTF8StringPointable.FACTORY),
-                                        PointableBinaryHashFunctionFactory
-                                                .of(UTF8StringPointable.FACTORY) }),
-                        tableSize), true);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
+                                new AvgFieldMergeAggregatorFactory(4, false) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyAvgExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -1073,58 +778,39 @@
     public void multiKeyMinMaxStringInmemGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
-                spec, keyFields, new FieldHashPartitionComputerFactory(
-                        keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, false) }), outputRec, tableSize);
+        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true),
+                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyMinMaxStringInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringInmemGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -1137,51 +823,35 @@
     public void multiKeyMinMaxStringPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
 
-        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
-                spec, keyFields,
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, true),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, false) }), outputRec);
+        PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, true),
+                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyMinMaxStringPreClusterGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringPreClusterGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
@@ -1194,71 +864,44 @@
     public void multiKeyMinMaxStringExtGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
-                spec, splitProvider, tupleParserFactory, desc);
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-                csvScanner, NC2_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(
-                new ISerializerDeserializer[] {
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE,
-                        IntegerSerializerDeserializer.INSTANCE,
-                        UTF8StringSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int frameLimits = 4;
         int tableSize = 8;
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
-                spec,
-                keyFields,
-                frameLimits,
-                new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(1, false),
-                                new MinMaxStringFieldAggregatorFactory(15,
-                                        true, true) }),
-                new MultiFieldsAggregatorFactory(new int[] { 0, 1 },
-                        new IFieldAggregateDescriptorFactory[] {
-                                new IntSumFieldAggregatorFactory(2, false),
-                                new MinMaxStringFieldAggregatorFactory(3, true,
-                                        true) }),
-                outputRec,
-                new HashSpillableTableFactory(
-                        new FieldHashPartitionComputerFactory(
-                                keyFields,
-                                new IBinaryHashFunctionFactory[] {
-                                        PointableBinaryHashFunctionFactory
-                                                .of(UTF8StringPointable.FACTORY),
-                                        PointableBinaryHashFunctionFactory
-                                                .of(UTF8StringPointable.FACTORY) }),
-                        tableSize), true);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
-                NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
-                spec, new FieldHashPartitionComputerFactory(keyFields,
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new MinMaxStringFieldAggregatorFactory(15, true, true) }),
+                new MultiFieldsAggregatorFactory(new int[] { 0, 1 }, new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(2, false),
+                        new MinMaxStringFieldAggregatorFactory(3, true, true) }), outputRec,
+                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
                         new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory
-                                        .of(UTF8StringPointable.FACTORY) }));
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
-                "multiKeyMinMaxStringExtGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringExtGroupTest");
 
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
-                NC2_ID, NC1_ID);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn2, grouper, 0, printer, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
index aea6126..5008991 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
@@ -25,6 +25,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,13 +45,14 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class CountOfCountsTest extends AbstractIntegrationTest {
     @Test
@@ -76,12 +78,10 @@
 
         RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                new int[] { 0 },
+        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                desc2);
+                new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID);
 
         InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
@@ -91,13 +91,15 @@
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(
                         new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
@@ -148,12 +150,10 @@
 
         RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                new int[] { 0 },
+        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                desc2);
+                new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
         InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
@@ -163,13 +163,16 @@
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(
                         new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
+
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
@@ -220,12 +223,10 @@
 
         RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
                 UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                new int[] { 0 },
+        PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                desc2);
+                new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
         InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
@@ -235,13 +236,16 @@
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
         PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                new MultiFieldsAggregatorFactory(
                         new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
+
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
index 0d5a627..93ed2c7 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
@@ -26,8 +26,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
@@ -52,12 +52,13 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.FloatSumFieldAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class LocalityAwareConnectorTest extends AbstractMultiNCIntegrationTest {
 
@@ -200,13 +201,13 @@
         runTest(spec);
     }
 
-    private AbstractSingleActivityOperatorDescriptor getPrinter(IOperatorDescriptorRegistry spec, String prefix)
+    private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification spec, String prefix)
             throws IOException {
 
-        AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec,
-                new ConstantFileSplitProvider(new FileSplit[] {
-                        new FileSplit("asterix-005", createTempFile().getAbsolutePath()),
-                        new FileSplit("asterix-006", createTempFile().getAbsolutePath()) }), "\t");
+        ResultSetId rsId = new ResultSetId(1);
+        AbstractSingleActivityOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
 
         return printer;
     }
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
index 1ee4400..ec9be32 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -40,9 +41,10 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.LimitOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.OptimizedExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class OptimizedSortMergeTest extends AbstractIntegrationTest {
 
@@ -75,9 +77,11 @@
                         PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -128,9 +132,11 @@
         LimitOperatorDescriptor filter = new LimitOperatorDescriptor(spec, ordersDesc, outputLimit);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, filter, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
index 9355110..961f780 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
@@ -42,7 +43,8 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class ScanPrintTest extends AbstractIntegrationTest {
     @Test
@@ -63,10 +65,11 @@
                 desc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] {
-                new FileSplit(NC2_ID, createTempFile().getAbsolutePath()),
-                new FileSplit(NC1_ID, createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
+
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn = new OneToOneConnectorDescriptor(spec);
@@ -98,9 +101,11 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
+
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
@@ -135,9 +140,11 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        spec.addResultSetId(rsId);
+
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
index 2c3fddf..0da93f2 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -40,9 +41,10 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class SortMergeTest extends AbstractIntegrationTest {
     @Test
@@ -73,9 +75,11 @@
                 ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -118,9 +122,11 @@
                         PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java
index 2b32142..6040748 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java
@@ -26,6 +26,7 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -35,8 +36,9 @@
 import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.LineFileWriteOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.SplitOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class SplitOperatorTest extends AbstractIntegrationTest {
 
@@ -50,6 +52,8 @@
             Assert.assertEquals(lineA, lineB);
         }
         Assert.assertNull(fileB.readLine());
+        fileA.close();
+        fileB.close();
     }
 
     @Test
@@ -83,8 +87,11 @@
 
         IOperatorDescriptor outputOp[] = new IOperatorDescriptor[outputFile.length];
         for (int i = 0; i < outputArity; i++) {
-            outputOp[i] = new LineFileWriteOperatorDescriptor(spec, new FileSplit[] { new FileSplit(NC1_ID,
-                    outputFile[i].getAbsolutePath()) });
+            ResultSetId rsId = new ResultSetId(i);
+            spec.addResultSetId(rsId);
+
+            outputOp[i] = new ResultWriterOperatorDescriptor(spec, rsId, true,
+                    ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
             PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, outputOp[i], locations);
         }
 
@@ -96,10 +103,10 @@
         for (int i = 0; i < outputArity; i++) {
             spec.addRoot(outputOp[i]);
         }
-        runTest(spec);
-
+        String[] expectedResultsFileNames = new String[outputArity];
         for (int i = 0; i < outputArity; i++) {
-            compareFiles(inputFileName, outputFile[i].getAbsolutePath());
+            expectedResultsFileNames[i] = inputFileName;
         }
+        runTestAndCompareResults(spec, expectedResultsFileNames);
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index 622942b..b5eb850 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -14,9 +14,7 @@
  */
 package edu.uci.ics.hyracks.tests.integration;
 
-import java.io.DataOutput;
 import java.io.File;
-import java.io.IOException;
 
 import org.junit.Test;
 
@@ -25,11 +23,10 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -47,12 +44,13 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.GraceHashJoinOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.HybridHashJoinOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.InMemoryHashJoinOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.MaterializingOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.tests.util.NoopNullWriterFactory;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class TPCHCustomerOrderHashJoinTest extends AbstractIntegrationTest {
 
@@ -127,9 +125,11 @@
                 custOrderJoinDesc, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -208,9 +208,11 @@
                 custOrderJoinDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -289,9 +291,11 @@
                 custOrderJoinDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -371,9 +375,11 @@
                 custOrderJoinDesc, true, nullWriterFactories, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -457,9 +463,11 @@
                 custOrderJoinDesc, true, nullWriterFactories);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -543,9 +551,11 @@
                 custOrderJoinDesc, true, nullWriterFactories);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -622,9 +632,11 @@
                 custOrderJoinDesc, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -711,9 +723,11 @@
                 custOrderJoinDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -800,9 +814,11 @@
                 custOrderJoinDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -885,9 +901,11 @@
                 custOrderJoinDesc, 128);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -976,9 +994,11 @@
                 custOrderJoinDesc, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordPartConn = new MToNPartitioningConnectorDescriptor(spec,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
index 9233e39..99f2d18 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
@@ -15,7 +15,9 @@
 package edu.uci.ics.hyracks.tests.integration;
 
 import java.io.File;
+
 import org.junit.Test;
+
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -28,6 +30,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -42,9 +45,10 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.NestedLoopJoinOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.tests.util.NoopNullWriterFactory;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class TPCHCustomerOrderNestedLoopJoinTest extends AbstractIntegrationTest {
     private static class JoinComparatorFactory implements ITuplePairComparatorFactory {
@@ -169,9 +173,11 @@
                 null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -244,9 +250,11 @@
                 null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -319,9 +327,11 @@
                 null);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -399,9 +409,11 @@
                 nullWriterFactories);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java
index 37b55b8..5b323e6 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -33,8 +34,9 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.union.UnionAllOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class UnionTest extends AbstractIntegrationTest {
     @Test
@@ -65,10 +67,11 @@
         UnionAllOperatorDescriptor unionAll = new UnionAllOperatorDescriptor(spec, 2, desc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, unionAll, NC2_ID, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] {
-                new FileSplit(NC2_ID, createTempFile().getAbsolutePath()),
-                new FileSplit(NC1_ID, createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        ResultSetId rsId = new ResultSetId(1);
+        spec.addResultSetId(rsId);
+
+        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), csvScanner01, 0, unionAll, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/ResultSerializerFactoryProvider.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/ResultSerializerFactoryProvider.java
new file mode 100644
index 0000000..19c4475
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/ResultSerializerFactoryProvider.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.tests.util;
+
+import java.io.DataInputStream;
+import java.io.PrintStream;
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+public class ResultSerializerFactoryProvider implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public static final ResultSerializerFactoryProvider INSTANCE = new ResultSerializerFactoryProvider();
+
+    private ResultSerializerFactoryProvider() {
+    }
+
+    public IResultSerializerFactory getResultSerializerFactoryProvider() {
+        return new IResultSerializerFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IResultSerializer createResultSerializer(final RecordDescriptor recordDesc,
+                    final PrintStream printStream) {
+                return new IResultSerializer() {
+                    private static final long serialVersionUID = 1L;
+
+                    ByteBufferInputStream bbis = new ByteBufferInputStream();
+                    DataInputStream di = new DataInputStream(bbis);
+
+                    @Override
+                    public void init() throws HyracksDataException {
+
+                    }
+
+                    @Override
+                    public boolean appendTuple(IFrameTupleAccessor tAccess, int tIdx) throws HyracksDataException {
+                        int start = tAccess.getTupleStartOffset(tIdx) + tAccess.getFieldSlotsLength();
+
+                        bbis.setByteBuffer(tAccess.getBuffer(), start);
+
+                        Object[] record = new Object[recordDesc.getFieldCount()];
+                        for (int i = 0; i < record.length; ++i) {
+                            Object instance = recordDesc.getFields()[i].deserialize(di);
+                            if (i == 0) {
+                                printStream.print(String.valueOf(instance));
+                            } else {
+                                printStream.print(", " + String.valueOf(instance));
+                            }
+                        }
+                        printStream.println();
+                        return true;
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/hyracks/hyracks-examples/text-example/pom.xml b/hyracks/hyracks-examples/text-example/pom.xml
index 016897e..748371f 100644
--- a/hyracks/hyracks-examples/text-example/pom.xml
+++ b/hyracks/hyracks-examples/text-example/pom.xml
@@ -13,6 +13,6 @@
   <modules>
     <module>texthelper</module>
     <module>textclient</module>
-    <module>textapp</module>
+    <module>textserver</module>
   </modules>
 </project>
diff --git a/hyracks/hyracks-examples/text-example/textapp/src/main/assembly/app-assembly.xml b/hyracks/hyracks-examples/text-example/textapp/src/main/assembly/app-assembly.xml
deleted file mode 100644
index 43ace6c..0000000
--- a/hyracks/hyracks-examples/text-example/textapp/src/main/assembly/app-assembly.xml
+++ /dev/null
@@ -1,13 +0,0 @@
-<assembly>
-  <id>app-assembly</id>
-  <formats>
-    <format>zip</format>
-  </formats>
-  <includeBaseDirectory>false</includeBaseDirectory>
-  <fileSets>
-    <fileSet>
-      <directory>target/application/lib</directory>
-      <outputDirectory>lib</outputDirectory>
-    </fileSet>
-  </fileSets>
-</assembly>
diff --git a/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java b/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
index 943f232..18813a7 100644
--- a/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
+++ b/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
@@ -74,9 +74,6 @@
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
         public int port = 1098;
 
-        @Option(name = "-app", usage = "Hyracks Application name", required = true)
-        public String app;
-
         @Option(name = "-infile-splits", usage = "Comma separated list of file-splits for the input. A file-split is <node-name>:<path>", required = true)
         public String inFileSplits;
 
@@ -122,7 +119,7 @@
 
             System.out.print(i + "\t" + (System.currentTimeMillis() - start));
             start = System.currentTimeMillis();
-            JobId jobId = hcc.startJob(options.app, job);
+            JobId jobId = hcc.startJob(job);
             hcc.waitForCompletion(jobId);
             System.out.println("\t" + (System.currentTimeMillis() - start));
         }
diff --git a/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java b/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
index 31019ab..5b6fad9 100644
--- a/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
+++ b/hyracks/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
@@ -65,9 +65,6 @@
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
         public int port = 1098;
 
-        @Option(name = "-app", usage = "Hyracks Application name", required = true)
-        public String app;
-
         @Option(name = "-infile-splits", usage = "Comma separated list of file-splits for the input. A file-split is <node-name>:<path>", required = true)
         public String inFileSplits;
 
@@ -101,8 +98,8 @@
                 options.algo, options.htSize, options.sbSize, options.format);
 
         long start = System.currentTimeMillis();
-        JobId jobId = hcc.startJob(options.app, job, options.runtimeProfiling ? EnumSet.of(JobFlag.PROFILE_RUNTIME)
-                : EnumSet.noneOf(JobFlag.class));
+        JobId jobId = hcc.startJob(job,
+                options.runtimeProfiling ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
         System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks/hyracks-examples/text-example/textapp/data/file1.txt b/hyracks/hyracks-examples/text-example/textserver/data/file1.txt
similarity index 100%
rename from hyracks/hyracks-examples/text-example/textapp/data/file1.txt
rename to hyracks/hyracks-examples/text-example/textserver/data/file1.txt
diff --git a/hyracks/hyracks-examples/text-example/textapp/data/file2.txt b/hyracks/hyracks-examples/text-example/textserver/data/file2.txt
similarity index 100%
rename from hyracks/hyracks-examples/text-example/textapp/data/file2.txt
rename to hyracks/hyracks-examples/text-example/textserver/data/file2.txt
diff --git a/hyracks/hyracks-examples/text-example/textapp/pom.xml b/hyracks/hyracks-examples/text-example/textserver/pom.xml
similarity index 63%
rename from hyracks/hyracks-examples/text-example/textapp/pom.xml
rename to hyracks/hyracks-examples/text-example/textserver/pom.xml
index fe5fc25..1b1f2d4 100644
--- a/hyracks/hyracks-examples/text-example/textapp/pom.xml
+++ b/hyracks/hyracks-examples/text-example/textserver/pom.xml
@@ -1,58 +1,42 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
   <modelVersion>4.0.0</modelVersion>
-  <artifactId>textapp</artifactId>
-  <name>textapp</name>
+  <groupId>edu.uci.ics.hyracks.examples.text</groupId>
+  <artifactId>textserver</artifactId>
+  <version>0.2.3-SNAPSHOT</version>
+  <name>textserver</name>
 
   <parent>
-    <groupId>edu.uci.ics.hyracks</groupId>
+    <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>text-example</artifactId>
     <version>0.2.3-SNAPSHOT</version>
   </parent>
 
   <build>
-    <pluginManagement>
-      <plugins>
-        <plugin>
-          <groupId>org.eclipse.m2e</groupId>
-          <artifactId>lifecycle-mapping</artifactId>
-          <version>1.0.0</version>
-          <configuration>
-            <lifecycleMappingMetadata>
-              <pluginExecutions>
-                <pluginExecution>
-                  <pluginExecutionFilter>
-                    <groupId>org.apache.maven.plugins</groupId>
-                    <artifactId>maven-dependency-plugin</artifactId>
-                    <versionRange>[1.0.0,)</versionRange>
-                    <goals>
-                      <goal>copy-dependencies</goal>
-                    </goals>
-                  </pluginExecutionFilter>
-                  <action>
-                    <ignore />
-                  </action>
-                </pluginExecution>
-              </pluginExecutions>
-            </lifecycleMappingMetadata>
-          </configuration>
-        </plugin>
-      </plugins>
-	</pluginManagement>
-
     <plugins>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>appassembler-maven-plugin</artifactId>
+        <version>1.3</version>
         <executions>
           <execution>
-            <id>copy-dependencies</id>
+            <configuration>
+              <programs>
+                <program>
+                  <mainClass>edu.uci.ics.hyracks.control.cc.CCDriver</mainClass>
+                  <name>hyrackscc</name>
+                </program>
+                <program>
+                  <mainClass>edu.uci.ics.hyracks.control.nc.NCDriver</mainClass>
+                  <name>hyracksnc</name>
+                </program>
+              </programs>
+              <repositoryLayout>flat</repositoryLayout>
+              <repositoryName>lib</repositoryName>
+            </configuration>
             <phase>package</phase>
             <goals>
-              <goal>copy-dependencies</goal>
+              <goal>assemble</goal>
             </goals>
-            <configuration>
-              <outputDirectory>target/application/lib</outputDirectory>
-            </configuration>
           </execution>
         </executions>
       </plugin>
@@ -63,7 +47,7 @@
           <execution>
             <configuration>
               <descriptors>
-                <descriptor>src/main/assembly/app-assembly.xml</descriptor>
+                <descriptor>src/main/assembly/binary-assembly.xml</descriptor>
               </descriptors>
             </configuration>
             <phase>package</phase>
@@ -78,8 +62,7 @@
       	<artifactId>hyracks-virtualcluster-maven-plugin</artifactId>
       	<version>0.2.3-SNAPSHOT</version>
         <configuration>
-          <hyracksServerHome>${basedir}/../../../hyracks-server/target/hyracks-server-${project.version}-binary-assembly</hyracksServerHome>
-          <hyracksCLIHome>${basedir}/../../../hyracks-cli/target/hyracks-cli-${project.version}-binary-assembly</hyracksCLIHome>
+          <hyracksServerHome>${basedir}/target/textserver-${project.version}-binary-assembly</hyracksServerHome>
           <jvmOptions>${jvm.extraargs}</jvmOptions>
         </configuration>
         <executions>
@@ -115,18 +98,6 @@
             </configuration>
           </execution>
           <execution>
-            <id>deploy-app</id>
-            <phase>pre-integration-test</phase>
-            <goals>
-              <goal>deploy-app</goal>
-            </goals>
-            <configuration>
-              <ccHost>localhost</ccHost>
-              <appName>text</appName>
-              <harFile>${project.build.directory}/textapp-${project.version}-app-assembly.zip</harFile>
-            </configuration>
-          </execution>
-          <execution>
             <id>stop-services</id>
             <phase>post-integration-test</phase>
             <goals>
@@ -140,9 +111,8 @@
       	<artifactId>maven-compiler-plugin</artifactId>
       	<version>2.0.2</version>
         <configuration>
-          <source>1.7</source>
-          <target>1.7</target>
-          <fork>true</fork>
+          <source>1.6</source>
+          <target>1.6</target>
         </configuration>
       </plugin>
       <plugin>
@@ -163,13 +133,27 @@
   </build>
   <dependencies>
   	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<groupId>edu.uci.ics.hyracks.examples.text</groupId>
   		<artifactId>texthelper</artifactId>
   		<version>0.2.3-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-cc</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-nc</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks.examples.text</groupId>
   		<artifactId>textclient</artifactId>
   		<version>0.2.3-SNAPSHOT</version>
   		<type>jar</type>
diff --git a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml b/hyracks/hyracks-examples/text-example/textserver/src/main/assembly/binary-assembly.xml
similarity index 81%
copy from hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
copy to hyracks/hyracks-examples/text-example/textserver/src/main/assembly/binary-assembly.xml
index 0500499..cd598d9 100644
--- a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
+++ b/hyracks/hyracks-examples/text-example/textserver/src/main/assembly/binary-assembly.xml
@@ -15,5 +15,9 @@
       <directory>target/appassembler/lib</directory>
       <outputDirectory>lib</outputDirectory>
     </fileSet>
+    <fileSet>
+      <directory>docs</directory>
+      <outputDirectory>docs</outputDirectory>
+    </fileSet>
   </fileSets>
 </assembly>
diff --git a/hyracks/hyracks-examples/text-example/textapp/src/test/java/edu/uci/ics/hyracks/examples/text/test/WordCountIT.java b/hyracks/hyracks-examples/text-example/textserver/src/test/java/edu/uci/ics/hyracks/examples/text/test/WordCountIT.java
similarity index 90%
rename from hyracks/hyracks-examples/text-example/textapp/src/test/java/edu/uci/ics/hyracks/examples/text/test/WordCountIT.java
rename to hyracks/hyracks-examples/text-example/textserver/src/test/java/edu/uci/ics/hyracks/examples/text/test/WordCountIT.java
index 9659288..76af72d 100644
--- a/hyracks/hyracks-examples/text-example/textapp/src/test/java/edu/uci/ics/hyracks/examples/text/test/WordCountIT.java
+++ b/hyracks/hyracks-examples/text-example/textserver/src/test/java/edu/uci/ics/hyracks/examples/text/test/WordCountIT.java
@@ -10,7 +10,7 @@
     @Test
     public void runWordCount() throws Exception {
         WordCountMain.main(new String[] { "-host", "localhost", "-infile-splits", getInfileSplits(), "-outfile-splits",
-                getOutfileSplits(), "-algo", "-hash", "-app", "text" });
+                getOutfileSplits(), "-algo", "-hash" });
     }
 
     private String getInfileSplits() {
@@ -22,4 +22,4 @@
         return "NC1:" + new File("target/wc1.txt").getAbsolutePath() + ",NC2:"
                 + new File("target/wc2.txt").getAbsolutePath();
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-examples/tpch-example/pom.xml b/hyracks/hyracks-examples/tpch-example/pom.xml
index ccfe002..79053e8 100644
--- a/hyracks/hyracks-examples/tpch-example/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/pom.xml
@@ -12,6 +12,6 @@
 
   <modules>
     <module>tpchclient</module>
-    <module>tpchapp</module>
+    <module>tpchserver</module>
   </modules>
 </project>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchapp/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchapp/pom.xml
deleted file mode 100644
index 354cc16..0000000
--- a/hyracks/hyracks-examples/tpch-example/tpchapp/pom.xml
+++ /dev/null
@@ -1,90 +0,0 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <artifactId>tpchapp</artifactId>
-  <name>tpchapp</name>
-  <parent>
-    <groupId>edu.uci.ics.hyracks</groupId>
-    <artifactId>tpch-example</artifactId>
-    <version>0.2.3-SNAPSHOT</version>
-  </parent>
-
-  <build>
-    <pluginManagement>
-      <plugins>
-        <plugin>
-          <groupId>org.eclipse.m2e</groupId>
-          <artifactId>lifecycle-mapping</artifactId>
-          <version>1.0.0</version>
-          <configuration>
-            <lifecycleMappingMetadata>
-              <pluginExecutions>
-                <pluginExecution>
-                  <pluginExecutionFilter>
-                    <groupId>org.apache.maven.plugins</groupId>
-                    <artifactId>maven-dependency-plugin</artifactId>
-                    <versionRange>[1.0.0,)</versionRange>
-                    <goals>
-                      <goal>copy-dependencies</goal>
-                    </goals>
-                  </pluginExecutionFilter>
-                  <action>
-                    <ignore />
-                  </action>
-                </pluginExecution>
-              </pluginExecutions>
-            </lifecycleMappingMetadata>
-          </configuration>
-        </plugin>
-      </plugins>
-	</pluginManagement>
-
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>copy-dependencies</id>
-            <phase>package</phase>
-            <goals>
-              <goal>copy-dependencies</goal>
-            </goals>
-            <configuration>
-              <outputDirectory>target/application/lib</outputDirectory>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <artifactId>maven-assembly-plugin</artifactId>
-        <version>2.2-beta-5</version>
-        <executions>
-          <execution>
-            <configuration>
-              <descriptors>
-                <descriptor>src/main/assembly/app-assembly.xml</descriptor>
-              </descriptors>
-            </configuration>
-            <phase>package</phase>
-            <goals>
-              <goal>attached</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-  <dependencies>
-    <dependency>
-        <groupId>edu.uci.ics.hyracks</groupId>
-        <artifactId>hyracks-dataflow-std</artifactId>
-        <version>0.2.3-SNAPSHOT</version>
-        <scope>compile</scope>
-    </dependency>
-    <dependency>
-    	<groupId>edu.uci.ics.hyracks</groupId>
-    	<artifactId>hyracks-data-std</artifactId>
-    	<version>0.2.3-SNAPSHOT</version>
-    </dependency>
-  </dependencies>
-</project>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchapp/src/main/assembly/app-assembly.xml b/hyracks/hyracks-examples/tpch-example/tpchapp/src/main/assembly/app-assembly.xml
deleted file mode 100644
index 43ace6c..0000000
--- a/hyracks/hyracks-examples/tpch-example/tpchapp/src/main/assembly/app-assembly.xml
+++ /dev/null
@@ -1,13 +0,0 @@
-<assembly>
-  <id>app-assembly</id>
-  <formats>
-    <format>zip</format>
-  </formats>
-  <includeBaseDirectory>false</includeBaseDirectory>
-  <fileSets>
-    <fileSet>
-      <directory>target/application/lib</directory>
-      <outputDirectory>lib</outputDirectory>
-    </fileSet>
-  </fileSets>
-</assembly>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index 0ad0ff0..6df9ff8 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -72,9 +72,6 @@
         @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)", required = false)
         public int port = 1098;
 
-        @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;
 
@@ -127,7 +124,7 @@
                 options.graceFactor, options.memSize, options.tableSize, options.hasGroupBy);
 
         long start = System.currentTimeMillis();
-        JobId jobId = hcc.startJob(options.app, job,
+        JobId jobId = hcc.startJob(job,
                 options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
         hcc.waitForCompletion(jobId);
         long end = System.currentTimeMillis();
diff --git a/hyracks/hyracks-cli/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
similarity index 60%
copy from hyracks/hyracks-cli/pom.xml
copy to hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
index 280478d..94d897a 100644
--- a/hyracks/hyracks-cli/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
@@ -1,43 +1,19 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
   <modelVersion>4.0.0</modelVersion>
-  <artifactId>hyracks-cli</artifactId>
-  <name>hyracks-cli</name>
+  <groupId>edu.uci.ics.hyracks.examples.tpch</groupId>
+  <artifactId>tpchserver</artifactId>
+  <version>0.2.3-SNAPSHOT</version>
+  <name>tpchserver</name>
 
   <parent>
-    <groupId>edu.uci.ics.hyracks</groupId>
-    <artifactId>hyracks</artifactId>
+    <groupId>edu.uci.ics.hyracks.examples</groupId>
+    <artifactId>tpch-example</artifactId>
     <version>0.2.3-SNAPSHOT</version>
   </parent>
 
   <build>
     <plugins>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <version>2.0.2</version>
-        <configuration>
-          <source>1.7</source>
-          <target>1.7</target>
-          <fork>true</fork>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>javacc-maven-plugin</artifactId>
-        <version>2.6</version>
-        <executions>
-          <execution>
-            <id>javacc</id>
-            <goals>
-              <goal>javacc</goal>
-            </goals>
-            <configuration>
-              <isStatic>false</isStatic>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
         <version>1.3</version>
@@ -46,8 +22,12 @@
             <configuration>
               <programs>
                 <program>
-                  <mainClass>edu.uci.ics.hyracks.cli.Main</mainClass>
-                  <name>hyrackscli</name>
+                  <mainClass>edu.uci.ics.hyracks.control.cc.CCDriver</mainClass>
+                  <name>hyrackscc</name>
+                </program>
+                <program>
+                  <mainClass>edu.uci.ics.hyracks.control.nc.NCDriver</mainClass>
+                  <name>hyracksnc</name>
                 </program>
               </programs>
               <repositoryLayout>flat</repositoryLayout>
@@ -80,17 +60,29 @@
     </plugins>
   </build>
   <dependencies>
+    <dependency>
+        <groupId>edu.uci.ics.hyracks</groupId>
+        <artifactId>hyracks-dataflow-std</artifactId>
+        <version>0.2.3-SNAPSHOT</version>
+        <scope>compile</scope>
+    </dependency>
+    <dependency>
+    	<groupId>edu.uci.ics.hyracks</groupId>
+    	<artifactId>hyracks-data-std</artifactId>
+    	<version>0.2.3-SNAPSHOT</version>
+    </dependency>
   	<dependency>
-  		<groupId>jline</groupId>
-  		<artifactId>jline</artifactId>
-  		<version>0.9.94</version>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-cc</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-api</artifactId>
+  		<artifactId>hyracks-control-nc</artifactId>
   		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   </dependencies>
diff --git a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml b/hyracks/hyracks-examples/tpch-example/tpchserver/src/main/assembly/binary-assembly.xml
similarity index 81%
copy from hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
copy to hyracks/hyracks-examples/tpch-example/tpchserver/src/main/assembly/binary-assembly.xml
index 0500499..cd598d9 100644
--- a/hyracks/hyracks-cli/src/main/assembly/binary-assembly.xml
+++ b/hyracks/hyracks-examples/tpch-example/tpchserver/src/main/assembly/binary-assembly.xml
@@ -15,5 +15,9 @@
       <directory>target/appassembler/lib</directory>
       <outputDirectory>lib</outputDirectory>
     </fileSet>
+    <fileSet>
+      <directory>docs</directory>
+      <outputDirectory>docs</outputDirectory>
+    </fileSet>
   </fileSets>
 </assembly>
diff --git a/hyracks/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java b/hyracks/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
index dfd229a..89cc1e5 100644
--- a/hyracks/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
+++ b/hyracks/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
@@ -40,35 +40,19 @@
         initialize(properties);
     }
 
-    private Set<String> getRequiredLibs(Set<String> userLibs) {
-        Set<String> requiredLibs = new HashSet<String>();
-        for (String systemLib : systemLibs) {
-            requiredLibs.add(systemLib);
-        }
-        for (String userLib : userLibs) {
-            requiredLibs.add(userLib);
-        }
-        return requiredLibs;
-    }
-
     public JobStatus getJobStatus(JobId jobId) throws Exception {
         return connection.getJobStatus(jobId);
     }
 
-    private void createApplication(String applicationName, Set<String> userLibs) throws Exception {
-        connection.createApplication(applicationName,
-                Utilities.getHyracksArchive(applicationName, getRequiredLibs(userLibs)));
-    }
-
     public HyracksRunningJob submitJob(String applicationName, JobSpecification spec) throws Exception {
         String jobProfilingVal = System.getenv(jobProfilingKey);
         boolean doProfiling = ("true".equalsIgnoreCase(jobProfilingVal));
         JobId jobId;
         if (doProfiling) {
             System.out.println("PROFILING");
-            jobId = connection.startJob(applicationName, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+            jobId = connection.startJob(spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
         } else {
-            jobId = connection.startJob(applicationName, spec);
+            jobId = connection.startJob(spec);
         }
         HyracksRunningJob runningJob = new HyracksRunningJob(jobId, spec, this);
         return runningJob;
@@ -76,7 +60,6 @@
 
     public HyracksRunningJob submitJob(String applicationName, JobSpecification spec, Set<String> userLibs)
             throws Exception {
-        createApplication(applicationName, userLibs);
         return submitJob(applicationName, spec);
     }
 
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollection.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollection.java
new file mode 100644
index 0000000..c51c1dd
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollection.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.hyracks.hdfs.api;
+
+import org.apache.hadoop.mapred.InputSplit;
+
+@SuppressWarnings("deprecation")
+public interface INcCollection {
+
+    public String findNearestAvailableSlot(InputSplit split);
+
+    public int numAvailableSlots();
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollectionBuilder.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollectionBuilder.java
new file mode 100644
index 0000000..ef3ff23
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollectionBuilder.java
@@ -0,0 +1,18 @@
+package edu.uci.ics.hyracks.hdfs.api;
+
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+
+/**
+ * NC collections
+ * 
+ * @author yingyib
+ */
+public interface INcCollectionBuilder {
+
+    public INcCollection build(Map<String, NodeControllerInfo> ncNameToNcInfos,
+            Map<String, List<String>> ipToNcMapping, Map<String, Integer> ncNameToIndex, String[] NCs, int[] workloads,
+            int slotLimit);
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
index 36d5f55..3ce6b2a 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
@@ -93,6 +93,7 @@
                 try {
                     FileSystem dfs = FileSystem.get(conf);
                     dos = dfs.create(new Path(fileName), true);
+                    tupleWriter.open(dos);
                 } catch (Exception e) {
                     throw new HyracksDataException(e);
                 }
@@ -116,6 +117,7 @@
             @Override
             public void close() throws HyracksDataException {
                 try {
+                    tupleWriter.close(dos);
                     dos.close();
                 } catch (Exception e) {
                     throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java
index 9cc9ebc..147e872 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java
@@ -23,6 +23,7 @@
 import java.io.Serializable;
 import java.lang.reflect.Constructor;
 
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputSplit;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -38,6 +39,8 @@
         splitBytes = splitsToBytes(splits);
         if (splits.length > 0) {
             splitClassName = splits[0].getClass().getName();
+        } else {
+            splitClassName = FileSplit.class.getName();
         }
     }
 
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/IPProximityNcCollectionBuilder.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/IPProximityNcCollectionBuilder.java
new file mode 100644
index 0000000..320b48b
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/IPProximityNcCollectionBuilder.java
@@ -0,0 +1,121 @@
+package edu.uci.ics.hyracks.hdfs.scheduler;
+
+import java.net.InetAddress;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.hdfs.api.INcCollection;
+import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
+
+@SuppressWarnings("deprecation")
+public class IPProximityNcCollectionBuilder implements INcCollectionBuilder {
+
+    @Override
+    public INcCollection build(Map<String, NodeControllerInfo> ncNameToNcInfos,
+            final Map<String, List<String>> ipToNcMapping, final Map<String, Integer> ncNameToIndex, String[] NCs,
+            final int[] workloads, final int slotLimit) {
+        final TreeMap<BytesWritable, IntWritable> availableIpsToSlots = new TreeMap<BytesWritable, IntWritable>();
+        for (int i = 0; i < workloads.length; i++) {
+            if (workloads[i] < slotLimit) {
+                BytesWritable ip = new BytesWritable(ncNameToNcInfos.get(NCs[i]).getNetworkAddress().getIpAddress());
+                IntWritable availableSlot = availableIpsToSlots.get(ip);
+                if (availableSlot == null) {
+                    availableSlot = new IntWritable(slotLimit - workloads[i]);
+                    availableIpsToSlots.put(ip, availableSlot);
+                } else {
+                    availableSlot.set(slotLimit - workloads[i] + availableSlot.get());
+                }
+            }
+        }
+        return new INcCollection() {
+
+            @Override
+            public String findNearestAvailableSlot(InputSplit split) {
+                try {
+                    String[] locs = split.getLocations();
+                    int minDistance = Integer.MAX_VALUE;
+                    BytesWritable currentCandidateIp = null;
+                    if (locs == null || locs.length > 0) {
+                        for (int j = 0; j < locs.length; j++) {
+                            /**
+                             * get all the IP addresses from the name
+                             */
+                            InetAddress[] allIps = InetAddress.getAllByName(locs[j]);
+                            for (InetAddress ip : allIps) {
+                                BytesWritable splitIp = new BytesWritable(ip.getAddress());
+                                /**
+                                 * if the node controller exists
+                                 */
+                                BytesWritable candidateNcIp = availableIpsToSlots.floorKey(splitIp);
+                                if (candidateNcIp == null) {
+                                    candidateNcIp = availableIpsToSlots.ceilingKey(splitIp);
+                                }
+                                if (candidateNcIp != null) {
+                                    if (availableIpsToSlots.get(candidateNcIp).get() > 0) {
+                                        byte[] candidateIP = candidateNcIp.getBytes();
+                                        byte[] splitIP = splitIp.getBytes();
+                                        int candidateInt = candidateIP[0] << 24 | (candidateIP[1] & 0xFF) << 16
+                                                | (candidateIP[2] & 0xFF) << 8 | (candidateIP[3] & 0xFF);
+                                        int splitInt = splitIP[0] << 24 | (splitIP[1] & 0xFF) << 16
+                                                | (splitIP[2] & 0xFF) << 8 | (splitIP[3] & 0xFF);
+                                        int distance = Math.abs(candidateInt - splitInt);
+                                        if (minDistance > distance) {
+                                            minDistance = distance;
+                                            currentCandidateIp = candidateNcIp;
+                                        }
+                                    }
+                                }
+                            }
+                        }
+                    } else {
+                        for (Entry<BytesWritable, IntWritable> entry : availableIpsToSlots.entrySet()) {
+                            if (entry.getValue().get() > 0) {
+                                currentCandidateIp = entry.getKey();
+                                break;
+                            }
+                        }
+                    }
+
+                    if (currentCandidateIp != null) {
+                        /**
+                         * Update the entry of the selected IP
+                         */
+                        IntWritable availableSlot = availableIpsToSlots.get(currentCandidateIp);
+                        availableSlot.set(availableSlot.get() - 1);
+                        if (availableSlot.get() == 0) {
+                            availableIpsToSlots.remove(currentCandidateIp);
+                        }
+                        /**
+                         * Update the entry of the selected NC
+                         */
+                        List<String> dataLocations = ipToNcMapping.get(InetAddress.getByAddress(
+                                currentCandidateIp.getBytes()).getHostAddress());
+                        for (String nc : dataLocations) {
+                            int ncIndex = ncNameToIndex.get(nc);
+                            if (workloads[ncIndex] < slotLimit) {
+                                return nc;
+                            }
+                        }
+                    }
+                    /** not scheduled */
+                    return null;
+                } catch (Exception e) {
+                    throw new IllegalStateException(e);
+                }
+            }
+
+            @Override
+            public int numAvailableSlots() {
+                return availableIpsToSlots.size();
+            }
+
+        };
+    }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/RackAwareNcCollectionBuilder.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/RackAwareNcCollectionBuilder.java
new file mode 100644
index 0000000..2b9e899
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/RackAwareNcCollectionBuilder.java
@@ -0,0 +1,202 @@
+package edu.uci.ics.hyracks.hdfs.scheduler;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+import java.util.logging.Logger;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.topology.ClusterTopology;
+import edu.uci.ics.hyracks.hdfs.api.INcCollection;
+import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
+
+@SuppressWarnings("deprecation")
+public class RackAwareNcCollectionBuilder implements INcCollectionBuilder {
+    private static final Logger LOGGER = Logger.getLogger(RackAwareNcCollectionBuilder.class.getName());
+    private ClusterTopology topology;
+
+    public RackAwareNcCollectionBuilder(ClusterTopology topology) {
+        this.topology = topology;
+    }
+
+    @Override
+    public INcCollection build(Map<String, NodeControllerInfo> ncNameToNcInfos,
+            final Map<String, List<String>> ipToNcMapping, final Map<String, Integer> ncNameToIndex, String[] NCs,
+            final int[] workloads, final int slotLimit) {
+        try {
+            final Map<List<Integer>, List<String>> pathToNCs = new HashMap<List<Integer>, List<String>>();
+            for (int i = 0; i < NCs.length; i++) {
+                List<Integer> path = new ArrayList<Integer>();
+                String ipAddress = InetAddress.getByAddress(
+                        ncNameToNcInfos.get(NCs[i]).getNetworkAddress().getIpAddress()).getHostAddress();
+                topology.lookupNetworkTerminal(ipAddress, path);
+                List<String> ncs = pathToNCs.get(path);
+                if (ncs == null) {
+                    ncs = new ArrayList<String>();
+                    pathToNCs.put(path, ncs);
+                }
+                ncs.add(NCs[i]);
+            }
+
+            final TreeMap<List<Integer>, IntWritable> availableIpsToSlots = new TreeMap<List<Integer>, IntWritable>(
+                    new Comparator<List<Integer>>() {
+
+                        @Override
+                        public int compare(List<Integer> l1, List<Integer> l2) {
+                            int commonLength = Math.min(l1.size(), l2.size());
+                            for (int i = 0; i < commonLength; i++) {
+                                Integer value1 = l1.get(i);
+                                Integer value2 = l2.get(i);
+                                int cmp = value1 > value2 ? 1 : (value1 < value2 ? -1 : 0);
+                                if (cmp != 0) {
+                                    return cmp;
+                                }
+                            }
+                            return l1.size() > l2.size() ? 1 : (l1.size() < l2.size() ? -1 : 0);
+                        }
+
+                    });
+            for (int i = 0; i < workloads.length; i++) {
+                if (workloads[i] < slotLimit) {
+                    List<Integer> path = new ArrayList<Integer>();
+                    String ipAddress = InetAddress.getByAddress(
+                            ncNameToNcInfos.get(NCs[i]).getNetworkAddress().getIpAddress()).getHostAddress();
+                    topology.lookupNetworkTerminal(ipAddress, path);
+                    IntWritable availableSlot = availableIpsToSlots.get(path);
+                    if (availableSlot == null) {
+                        availableSlot = new IntWritable(slotLimit - workloads[i]);
+                        availableIpsToSlots.put(path, availableSlot);
+                    } else {
+                        availableSlot.set(slotLimit - workloads[i] + availableSlot.get());
+                    }
+                }
+            }
+            return new INcCollection() {
+
+                @Override
+                public String findNearestAvailableSlot(InputSplit split) {
+                    try {
+                        String[] locs = split.getLocations();
+                        int minDistance = Integer.MAX_VALUE;
+                        List<Integer> currentCandidatePath = null;
+                        if (locs == null || locs.length > 0) {
+                            for (int j = 0; j < locs.length; j++) {
+                                /**
+                                 * get all the IP addresses from the name
+                                 */
+                                InetAddress[] allIps = InetAddress.getAllByName(locs[j]);
+                                boolean inTopology = false;
+                                for (InetAddress ip : allIps) {
+                                    List<Integer> splitPath = new ArrayList<Integer>();
+                                    boolean inCluster = topology.lookupNetworkTerminal(ip.getHostAddress(), splitPath);
+                                    if (!inCluster) {
+                                        continue;
+                                    }
+                                    inTopology = true;
+                                    /**
+                                     * if the node controller exists
+                                     */
+                                    List<Integer> candidatePath = availableIpsToSlots.floorKey(splitPath);
+                                    if (candidatePath == null) {
+                                        candidatePath = availableIpsToSlots.ceilingKey(splitPath);
+                                    }
+                                    if (candidatePath != null) {
+                                        if (availableIpsToSlots.get(candidatePath).get() > 0) {
+                                            int distance = distance(splitPath, candidatePath);
+                                            if (minDistance > distance) {
+                                                minDistance = distance;
+                                                currentCandidatePath = candidatePath;
+                                            }
+                                        }
+
+                                    }
+                                }
+
+                                if (!inTopology) {
+                                    LOGGER.info(locs[j] + "'s IP address is not in the cluster toplogy file!");
+                                    /**
+                                     * if the machine is not in the toplogy file
+                                     */
+                                    List<Integer> candidatePath = null;
+                                    for (Entry<List<Integer>, IntWritable> entry : availableIpsToSlots.entrySet()) {
+                                        if (entry.getValue().get() > 0) {
+                                            candidatePath = entry.getKey();
+                                            break;
+                                        }
+                                    }
+                                    /** the split path is empty */
+                                    if (candidatePath != null) {
+                                        if (availableIpsToSlots.get(candidatePath).get() > 0) {
+                                            currentCandidatePath = candidatePath;
+                                        }
+                                    }
+                                }
+                            }
+                        } else {
+                            for (Entry<List<Integer>, IntWritable> entry : availableIpsToSlots.entrySet()) {
+                                if (entry.getValue().get() > 0) {
+                                    currentCandidatePath = entry.getKey();
+                                    break;
+                                }
+                            }
+                        }
+
+                        if (currentCandidatePath != null && currentCandidatePath.size() > 0) {
+                            /**
+                             * Update the entry of the selected IP
+                             */
+                            IntWritable availableSlot = availableIpsToSlots.get(currentCandidatePath);
+                            availableSlot.set(availableSlot.get() - 1);
+                            if (availableSlot.get() == 0) {
+                                availableIpsToSlots.remove(currentCandidatePath);
+                            }
+                            /**
+                             * Update the entry of the selected NC
+                             */
+                            List<String> candidateNcs = pathToNCs.get(currentCandidatePath);
+                            for (String candidate : candidateNcs) {
+                                int ncIndex = ncNameToIndex.get(candidate);
+                                if (workloads[ncIndex] < slotLimit) {
+                                    return candidate;
+                                }
+                            }
+                        }
+                        /** not scheduled */
+                        return null;
+                    } catch (Exception e) {
+                        throw new IllegalStateException(e);
+                    }
+                }
+
+                @Override
+                public int numAvailableSlots() {
+                    return availableIpsToSlots.size();
+                }
+
+                private int distance(List<Integer> splitPath, List<Integer> candidatePath) {
+                    int commonLength = Math.min(splitPath.size(), candidatePath.size());
+                    int distance = 0;
+                    for (int i = 0; i < commonLength; i++) {
+                        distance = distance * 100 + Math.abs(splitPath.get(i) - candidatePath.get(i));
+                    }
+                    List<Integer> restElements = splitPath.size() > candidatePath.size() ? splitPath : candidatePath;
+                    for (int i = commonLength; i < restElements.size(); i++) {
+                        distance = distance * 100 + Math.abs(restElements.get(i));
+                    }
+                    return distance;
+                }
+            };
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java
index 3f287cf..3f7997b 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java
@@ -20,11 +20,15 @@
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.PriorityQueue;
 import java.util.Random;
+import java.util.logging.Logger;
 
+import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.mapred.InputSplit;
 
 import edu.uci.ics.hyracks.api.client.HyracksConnection;
@@ -32,13 +36,17 @@
 import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.topology.ClusterTopology;
+import edu.uci.ics.hyracks.hdfs.api.INcCollection;
+import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
 
 /**
- * The scheduler conduct data-local scheduling for data reading on HDFS.
- * This class works for Hadoop old API.
+ * The scheduler conduct data-local scheduling for data reading on HDFS. This
+ * class works for Hadoop old API.
  */
 @SuppressWarnings("deprecation")
 public class Scheduler {
+    private static final Logger LOGGER = Logger.getLogger(Scheduler.class.getName());
 
     /** a list of NCs */
     private String[] NCs;
@@ -49,6 +57,14 @@
     /** a map from the NC name to the index */
     private Map<String, Integer> ncNameToIndex = new HashMap<String, Integer>();
 
+    /** a map from NC name to the NodeControllerInfo */
+    private Map<String, NodeControllerInfo> ncNameToNcInfos;
+
+    /**
+     * the nc collection builder
+     */
+    private INcCollectionBuilder ncCollectionBuilder;
+
     /**
      * The constructor of the scheduler.
      * 
@@ -58,7 +74,10 @@
     public Scheduler(String ipAddress, int port) throws HyracksException {
         try {
             IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
-            Map<String, NodeControllerInfo> ncNameToNcInfos = hcc.getNodeControllerInfos();
+            this.ncNameToNcInfos = hcc.getNodeControllerInfos();
+            ClusterTopology topology = hcc.getClusterTopology();
+            this.ncCollectionBuilder = topology == null ? new IPProximityNcCollectionBuilder()
+                    : new RackAwareNcCollectionBuilder(topology);
             loadIPAddressToNCMap(ncNameToNcInfos);
         } catch (Exception e) {
             throw new HyracksException(e);
@@ -68,56 +87,122 @@
     /**
      * The constructor of the scheduler.
      * 
-     * @param ncNameToNcInfos the mapping from nc names to nc infos
+     * @param ncNameToNcInfos
+     * @throws HyracksException
+     */
+    public Scheduler(String ipAddress, int port, INcCollectionBuilder ncCollectionBuilder) throws HyracksException {
+        try {
+            IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
+            this.ncNameToNcInfos = hcc.getNodeControllerInfos();
+            this.ncCollectionBuilder = ncCollectionBuilder;
+            loadIPAddressToNCMap(ncNameToNcInfos);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    /**
+     * The constructor of the scheduler.
+     * 
+     * @param ncNameToNcInfos
+     *            the mapping from nc names to nc infos
      * @throws HyracksException
      */
     public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
+        this.ncNameToNcInfos = ncNameToNcInfos;
+        this.ncCollectionBuilder = new IPProximityNcCollectionBuilder();
         loadIPAddressToNCMap(ncNameToNcInfos);
     }
 
     /**
-     * Set location constraints for a file scan operator with a list of file splits.
-     * It guarantees the maximum slots a machine can is at most one more than the minimum slots a
-     * machine can get.
+     * The constructor of the scheduler.
+     * 
+     * @param ncNameToNcInfos
+     *            the mapping from nc names to nc infos
+     * @param topology
+     *            the hyracks cluster toplogy
+     * @throws HyracksException
+     */
+    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, ClusterTopology topology) throws HyracksException {
+        this(ncNameToNcInfos);
+        this.ncCollectionBuilder = topology == null ? new IPProximityNcCollectionBuilder()
+                : new RackAwareNcCollectionBuilder(topology);
+    }
+
+    /**
+     * The constructor of the scheduler.
+     * 
+     * @param ncNameToNcInfos
+     *            the mapping from nc names to nc infos
+     * @throws HyracksException
+     */
+    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, INcCollectionBuilder ncCollectionBuilder)
+            throws HyracksException {
+        this.ncNameToNcInfos = ncNameToNcInfos;
+        this.ncCollectionBuilder = ncCollectionBuilder;
+        loadIPAddressToNCMap(ncNameToNcInfos);
+    }
+
+    /**
+     * Set location constraints for a file scan operator with a list of file
+     * splits. It guarantees the maximum slots a machine can is at most one more
+     * than the minimum slots a machine can get.
      * 
      * @throws HyracksDataException
      */
     public String[] getLocationConstraints(InputSplit[] splits) throws HyracksException {
-        int[] capacity = new int[NCs.length];
-        Arrays.fill(capacity, 0);
+        if (splits == null) {
+            /** deal the case when the splits array is null */
+            return new String[] {};
+        }
+        int[] workloads = new int[NCs.length];
+        Arrays.fill(workloads, 0);
         String[] locations = new String[splits.length];
+        Map<String, IntWritable> locationToNumOfSplits = new HashMap<String, IntWritable>();
         /**
          * upper bound number of slots that a machine can get
          */
-        int upperBoundSlots = splits.length % capacity.length == 0 ? (splits.length / capacity.length) : (splits.length
-                / capacity.length + 1);
+        int upperBoundSlots = splits.length % workloads.length == 0 ? (splits.length / workloads.length)
+                : (splits.length / workloads.length + 1);
         /**
          * lower bound number of slots that a machine can get
          */
-        int lowerBoundSlots = splits.length % capacity.length == 0 ? upperBoundSlots : upperBoundSlots - 1;
+        int lowerBoundSlots = splits.length % workloads.length == 0 ? upperBoundSlots : upperBoundSlots - 1;
 
         try {
             Random random = new Random(System.currentTimeMillis());
             boolean scheduled[] = new boolean[splits.length];
             Arrays.fill(scheduled, false);
-
+            /**
+             * scan the splits and build the popularity map
+             * give the machines with less local splits more scheduling priority
+             */
+            buildPopularityMap(splits, locationToNumOfSplits);
             /**
              * push data-local lower-bounds slots to each machine
              */
-            scheduleLocalSlots(splits, capacity, locations, lowerBoundSlots, random, scheduled);
+            scheduleLocalSlots(splits, workloads, locations, lowerBoundSlots, random, scheduled, locationToNumOfSplits);
             /**
              * push data-local upper-bounds slots to each machine
              */
-            scheduleLocalSlots(splits, capacity, locations, upperBoundSlots, random, scheduled);
+            scheduleLocalSlots(splits, workloads, locations, upperBoundSlots, random, scheduled, locationToNumOfSplits);
 
+            int dataLocalCount = 0;
+            for (int i = 0; i < scheduled.length; i++) {
+                if (scheduled[i] == true) {
+                    dataLocalCount++;
+                }
+            }
+            LOGGER.info("Data local rate: "
+                    + (scheduled.length == 0 ? 0.0 : ((float) dataLocalCount / (float) (scheduled.length))));
             /**
              * push non-data-local lower-bounds slots to each machine
              */
-            scheduleNoLocalSlots(splits, capacity, locations, lowerBoundSlots, scheduled);
+            scheduleNonLocalSlots(splits, workloads, locations, lowerBoundSlots, scheduled);
             /**
              * push non-data-local upper-bounds slots to each machine
              */
-            scheduleNoLocalSlots(splits, capacity, locations, upperBoundSlots, scheduled);
+            scheduleNonLocalSlots(splits, workloads, locations, upperBoundSlots, scheduled);
             return locations;
         } catch (IOException e) {
             throw new HyracksException(e);
@@ -129,46 +214,38 @@
      * 
      * @param splits
      *            The HDFS file splits.
-     * @param capacity
+     * @param workloads
      *            The current capacity of each machine.
      * @param locations
      *            The result schedule.
-     * @param slots
+     * @param slotLimit
      *            The maximum slots of each machine.
      * @param scheduled
      *            Indicate which slot is scheduled.
      */
-    private void scheduleNoLocalSlots(InputSplit[] splits, int[] capacity, String[] locations, int slots,
-            boolean[] scheduled) {
+    private void scheduleNonLocalSlots(InputSplit[] splits, int[] workloads, String[] locations, int slotLimit,
+            boolean[] scheduled) throws IOException, UnknownHostException {
         /**
-         * find the lowest index the current available NCs
+         * build the map from available ips to the number of available slots
          */
-        int currentAvailableNC = 0;
-        for (int i = 0; i < capacity.length; i++) {
-            if (capacity[i] < slots) {
-                currentAvailableNC = i;
-                break;
-            }
+        INcCollection ncCollection = this.ncCollectionBuilder.build(ncNameToNcInfos, ipToNcMapping, ncNameToIndex, NCs,
+                workloads, slotLimit);
+        if (ncCollection.numAvailableSlots() == 0) {
+            return;
         }
-
         /**
          * schedule no-local file reads
          */
         for (int i = 0; i < splits.length; i++) {
-            // if there is no data-local NC choice, choose a random one
+            /** if there is no data-local NC choice, choose a random one */
             if (!scheduled[i]) {
-                locations[i] = NCs[currentAvailableNC];
-                capacity[currentAvailableNC]++;
-                scheduled[i] = true;
-
-                /**
-                 * move the available NC cursor to the next one
-                 */
-                for (int j = currentAvailableNC; j < capacity.length; j++) {
-                    if (capacity[j] < slots) {
-                        currentAvailableNC = j;
-                        break;
-                    }
+                InputSplit split = splits[i];
+                String selectedNcName = ncCollection.findNearestAvailableSlot(split);
+                if (selectedNcName != null) {
+                    int ncIndex = ncNameToIndex.get(selectedNcName);
+                    workloads[ncIndex]++;
+                    scheduled[i] = true;
+                    locations[i] = selectedNcName;
                 }
             }
         }
@@ -179,7 +256,7 @@
      * 
      * @param splits
      *            The HDFS file splits.
-     * @param capacity
+     * @param workloads
      *            The current capacity of each machine.
      * @param locations
      *            The result schedule.
@@ -192,19 +269,37 @@
      * @throws IOException
      * @throws UnknownHostException
      */
-    private void scheduleLocalSlots(InputSplit[] splits, int[] capacity, String[] locations, int slots, Random random,
-            boolean[] scheduled) throws IOException, UnknownHostException {
+    private void scheduleLocalSlots(InputSplit[] splits, int[] workloads, String[] locations, int slots, Random random,
+            boolean[] scheduled, final Map<String, IntWritable> locationToNumSplits) throws IOException,
+            UnknownHostException {
+        /** scheduling candidates will be ordered inversely according to their popularity */
+        PriorityQueue<String> scheduleCadndiates = new PriorityQueue<String>(3, new Comparator<String>() {
+
+            @Override
+            public int compare(String s1, String s2) {
+                return locationToNumSplits.get(s1).compareTo(locationToNumSplits.get(s2));
+            }
+
+        });
         for (int i = 0; i < splits.length; i++) {
+            if (scheduled[i]) {
+                continue;
+            }
             /**
              * get the location of all the splits
              */
-            String[] loc = splits[i].getLocations();
-            if (loc.length > 0) {
-                for (int j = 0; j < loc.length; j++) {
+            String[] locs = splits[i].getLocations();
+            if (locs.length > 0) {
+                scheduleCadndiates.clear();
+                for (int j = 0; j < locs.length; j++) {
+                    scheduleCadndiates.add(locs[j]);
+                }
+
+                for (String candidate : scheduleCadndiates) {
                     /**
                      * get all the IP addresses from the name
                      */
-                    InetAddress[] allIps = InetAddress.getAllByName(loc[j]);
+                    InetAddress[] allIps = InetAddress.getAllByName(candidate);
                     /**
                      * iterate overa all ips
                      */
@@ -223,16 +318,17 @@
                             /**
                              * check if the node is already full
                              */
-                            if (capacity[pos] < slots) {
+                            if (workloads[pos] < slots) {
                                 locations[i] = nc;
-                                capacity[pos]++;
+                                workloads[pos]++;
                                 scheduled[i] = true;
+                                break;
                             }
                         }
                     }
-
                     /**
-                     * break the loop for data-locations if the schedule has already been found
+                     * break the loop for data-locations if the schedule has
+                     * already been found
                      */
                     if (scheduled[i] == true) {
                         break;
@@ -243,6 +339,30 @@
     }
 
     /**
+     * Scan the splits once and build a popularity map
+     * 
+     * @param splits
+     *            the split array
+     * @param locationToNumOfSplits
+     *            the map to be built
+     * @throws IOException
+     */
+    private void buildPopularityMap(InputSplit[] splits, Map<String, IntWritable> locationToNumOfSplits)
+            throws IOException {
+        for (InputSplit split : splits) {
+            String[] locations = split.getLocations();
+            for (String loc : locations) {
+                IntWritable locCount = locationToNumOfSplits.get(loc);
+                if (locCount == null) {
+                    locCount = new IntWritable(0);
+                    locationToNumOfSplits.put(loc, locCount);
+                }
+                locCount.set(locCount.get() + 1);
+            }
+        }
+    }
+
+    /**
      * Load the IP-address-to-NC map from the NCNameToNCInfoMap
      * 
      * @param ncNameToNcInfos
@@ -251,6 +371,8 @@
     private void loadIPAddressToNCMap(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
         try {
             NCs = new String[ncNameToNcInfos.size()];
+            ipToNcMapping.clear();
+            ncNameToIndex.clear();
             int i = 0;
 
             /**
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
index 86ee527..c1c227c 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
@@ -39,8 +39,8 @@
 import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
 
 /**
- * The HDFS file write operator using the Hadoop new API.
- * To use this operator, a user need to provide an ITupleWriterFactory.
+ * The HDFS file write operator using the Hadoop new API. To use this operator,
+ * a user need to provide an ITupleWriterFactory.
  */
 public class HDFSWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
 
@@ -92,6 +92,7 @@
                 try {
                     FileSystem dfs = FileSystem.get(conf.getConfiguration());
                     dos = dfs.create(new Path(fileName), true);
+                    tupleWriter.open(dos);
                 } catch (Exception e) {
                     throw new HyracksDataException(e);
                 }
@@ -115,6 +116,7 @@
             @Override
             public void close() throws HyracksDataException {
                 try {
+                    tupleWriter.close(dos);
                     dos.close();
                 } catch (Exception e) {
                     throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java
index cb97ca1..cde187d 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java
@@ -23,6 +23,7 @@
 import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
 
 /**
  * The scheduler conduct data-local scheduling for data reading on HDFS.
@@ -53,6 +54,17 @@
     public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
         scheduler = new edu.uci.ics.hyracks.hdfs.scheduler.Scheduler(ncNameToNcInfos);
     }
+    
+    /**
+     * The constructor of the scheduler.
+     * 
+     * @param ncNameToNcInfos
+     *            the mapping from nc names to nc infos
+     * @throws HyracksException
+     */
+    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, INcCollectionBuilder builder) throws HyracksException {
+        scheduler = new edu.uci.ics.hyracks.hdfs.scheduler.Scheduler(ncNameToNcInfos, builder);
+    }
 
     /**
      * Set location constraints for a file scan operator with a list of file splits
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
index 2686077..affe1347 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
@@ -71,7 +71,6 @@
     private static final String HDFS_INPUT_PATH = "/customer/";
     private static final String HDFS_OUTPUT_PATH = "/customer_result/";
 
-    private static final String HYRACKS_APP_NAME = "DataflowTest";
     private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
     private MiniDFSCluster dfsCluster;
 
@@ -82,7 +81,6 @@
     public void setUp() throws Exception {
         cleanupStores();
         HyracksUtils.init();
-        HyracksUtils.createApp(HYRACKS_APP_NAME);
         FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
         FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
         startHDFS();
@@ -164,7 +162,7 @@
 
         IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
                 HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
-        JobId jobId = client.startJob(HYRACKS_APP_NAME, jobSpec);
+        JobId jobId = client.startJob(jobSpec);
         client.waitForCompletion(jobId);
 
         Assert.assertEquals(true, checkResults());
@@ -196,7 +194,6 @@
 
     @Override
     public void tearDown() throws Exception {
-        HyracksUtils.destroyApp(HYRACKS_APP_NAME);
         HyracksUtils.deinit();
         cleanupHDFS();
     }
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java
index c6b0416..1f394a9 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java
@@ -15,6 +15,8 @@
 
 package edu.uci.ics.hyracks.hdfs.scheduler;
 
+import java.io.FileReader;
+import java.io.IOException;
 import java.net.InetAddress;
 import java.util.HashMap;
 import java.util.Map;
@@ -25,13 +27,28 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputSplit;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
 
 import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
 import edu.uci.ics.hyracks.api.client.NodeStatus;
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.topology.ClusterTopology;
+import edu.uci.ics.hyracks.api.topology.TopologyDefinitionParser;
 
 @SuppressWarnings("deprecation")
 public class SchedulerTest extends TestCase {
+    private static String TOPOLOGY_PATH = "src/test/resources/topology.xml";
+
+    private ClusterTopology parseTopology() throws IOException, SAXException {
+        FileReader fr = new FileReader(TOPOLOGY_PATH);
+        InputSource in = new InputSource(fr);
+        try {
+            return TopologyDefinitionParser.parse(in);
+        } finally {
+            fr.close();
+        }
+    }
 
     /**
      * Test the scheduler for the case when the Hyracks cluster is the HDFS cluster
@@ -41,17 +58,23 @@
     public void testSchedulerSimple() throws Exception {
         Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
         ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.1").getAddress(), 5099)));
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.2").getAddress(), 5099)));
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.3").getAddress(), 5099)));
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.4").getAddress(), 5099)));
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.5").getAddress(), 5099)));
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.6").getAddress(), 5099)));
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
 
         InputSplit[] fileSplits = new InputSplit[6];
         fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
@@ -61,11 +84,17 @@
         fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
         fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
 
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc6", "nc2", "nc3", "nc5" };
+
         Scheduler scheduler = new Scheduler(ncNameToNcInfos);
         String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
 
-        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc5", "nc6" };
-
+        ClusterTopology topology = parseTopology();
+        scheduler = new Scheduler(ncNameToNcInfos, topology);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
         for (int i = 0; i < locationConstraints.length; i++) {
             Assert.assertEquals(locationConstraints[i], expectedResults[i]);
         }
@@ -79,17 +108,23 @@
     public void testSchedulerLargerHDFS() throws Exception {
         Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
         ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.1").getAddress(), 5099)));
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.2").getAddress(), 5099)));
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.3").getAddress(), 5099)));
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.4").getAddress(), 5099)));
-        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.5").getAddress(), 5099)));
-        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.6").getAddress(), 5099)));
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc7", new NodeControllerInfo("nc7", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.7").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc12", new NodeControllerInfo("nc12", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.12").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
 
         InputSplit[] fileSplits = new InputSplit[12];
         fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
@@ -100,17 +135,25 @@
         fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
         fileSplits[6] = new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
         fileSplits[7] = new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
-        fileSplits[8] = new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" });
+        fileSplits[8] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.14", "10.0.0.11", "10.0.0.13" });
         fileSplits[9] = new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" });
         fileSplits[10] = new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.7" });
-        fileSplits[11] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+        fileSplits[11] = new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" });
 
         Scheduler scheduler = new Scheduler(ncNameToNcInfos);
         String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
 
-        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc3", "nc2", "nc1", "nc4", "nc5", "nc6",
-                "nc6", "nc5" };
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc12",
+                "nc7", "nc7", "nc12" };
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
 
+        expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc7", "nc12", "nc7",
+                "nc12" };
+        ClusterTopology topology = parseTopology();
+        scheduler = new Scheduler(ncNameToNcInfos, topology);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
         for (int i = 0; i < locationConstraints.length; i++) {
             Assert.assertEquals(locationConstraints[i], expectedResults[i]);
         }
@@ -124,17 +167,23 @@
     public void testSchedulerSmallerHDFS() throws Exception {
         Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
         ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.1").getAddress(), 5099)));
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.2").getAddress(), 5099)));
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.3").getAddress(), 5099)));
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.4").getAddress(), 5099)));
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.5").getAddress(), 5099)));
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.6").getAddress(), 5099)));
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
 
         InputSplit[] fileSplits = new InputSplit[12];
         fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
@@ -150,12 +199,19 @@
         fileSplits[10] = new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
         fileSplits[11] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
 
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc5", "nc6",
+                "nc5", "nc6" };
+
         Scheduler scheduler = new Scheduler(ncNameToNcInfos);
         String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
 
-        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc3", "nc2", "nc1", "nc4", "nc5", "nc6",
-                "nc5", "nc6" };
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
 
+        ClusterTopology topology = parseTopology();
+        scheduler = new Scheduler(ncNameToNcInfos, topology);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
         for (int i = 0; i < locationConstraints.length; i++) {
             Assert.assertEquals(locationConstraints[i], expectedResults[i]);
         }
@@ -169,17 +225,23 @@
     public void testSchedulerSmallerHDFSOdd() throws Exception {
         Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
         ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.1").getAddress(), 5099)));
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.2").getAddress(), 5099)));
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.3").getAddress(), 5099)));
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.4").getAddress(), 5099)));
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.5").getAddress(), 5099)));
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.6").getAddress(), 5099)));
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
 
         InputSplit[] fileSplits = new InputSplit[13];
         fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
@@ -196,15 +258,84 @@
         fileSplits[11] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
         fileSplits[12] = new FileSplit(new Path("part-13"), 0, 0, new String[] { "10.0.0.2", "10.0.0.4", "10.0.0.5" });
 
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc5", "nc1",
+                "nc5", "nc2", "nc4" };
+
         Scheduler scheduler = new Scheduler(ncNameToNcInfos);
         String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
 
-        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc3", "nc2", "nc1", "nc4", "nc5", "nc6",
-                "nc5", "nc5", "nc6" };
-
         for (int i = 0; i < locationConstraints.length; i++) {
             Assert.assertEquals(locationConstraints[i], expectedResults[i]);
         }
+
+        ClusterTopology topology = parseTopology();
+        scheduler = new Scheduler(ncNameToNcInfos, topology);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+    }
+
+    /**
+     * Test boundary cases where splits array is empty or null
+     * 
+     * @throws Exception
+     */
+    public void testSchedulercBoundary() throws Exception {
+        Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+        ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
+        ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
+
+        /** test empty file splits */
+        InputSplit[] fileSplits = new InputSplit[0];
+        String[] expectedResults = new String[] {};
+
+        Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+        String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+        ClusterTopology topology = parseTopology();
+        scheduler = new Scheduler(ncNameToNcInfos, topology);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+        fileSplits = null;
+        expectedResults = new String[] {};
+
+        scheduler = new Scheduler(ncNameToNcInfos);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
+        scheduler = new Scheduler(ncNameToNcInfos, topology);
+        locationConstraints = scheduler.getLocationConstraints(fileSplits);
+        for (int i = 0; i < locationConstraints.length; i++) {
+            Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+        }
+
     }
 
 }
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
index d44b75a..bdff2fd 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
@@ -64,6 +64,7 @@
         ncConfig1.clusterNetIPAddress = "localhost";
         ncConfig1.ccPort = TEST_HYRACKS_CC_PORT;
         ncConfig1.dataIPAddress = "127.0.0.1";
+        ncConfig1.datasetIPAddress = "127.0.0.1";
         ncConfig1.nodeId = NC1_ID;
         nc1 = new NodeControllerService(ncConfig1);
         nc1.start();
@@ -73,6 +74,7 @@
         ncConfig2.clusterNetIPAddress = "localhost";
         ncConfig2.ccPort = TEST_HYRACKS_CC_PORT;
         ncConfig2.dataIPAddress = "127.0.0.1";
+        ncConfig2.datasetIPAddress = "127.0.0.1";
         ncConfig2.nodeId = NC2_ID;
         nc2 = new NodeControllerService(ncConfig2);
         nc2.start();
@@ -81,14 +83,6 @@
         hcc = new HyracksConnection(CC_HOST, TEST_HYRACKS_CC_CLIENT_PORT);
     }
 
-    public static void destroyApp(String hyracksAppName) throws Exception {
-        hcc.destroyApplication(hyracksAppName);
-    }
-
-    public static void createApp(String hyracksAppName) throws Exception {
-        hcc.createApplication(hyracksAppName, null);
-    }
-
     public static void deinit() throws Exception {
         nc2.stop();
         nc1.stop();
@@ -97,7 +91,7 @@
 
     public static void runJob(JobSpecification spec, String appName) throws Exception {
         spec.setFrameSize(FRAME_SIZE);
-        JobId jobId = hcc.startJob(appName, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+        JobId jobId = hcc.startJob(spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
         hcc.waitForCompletion(jobId);
     }
 
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
index 9f77979..c2892e9 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
@@ -74,7 +74,6 @@
     private static final String HDFS_INPUT_PATH = "/customer/";
     private static final String HDFS_OUTPUT_PATH = "/customer_result/";
 
-    private static final String HYRACKS_APP_NAME = "DataflowTest";
     private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
     private MiniDFSCluster dfsCluster;
     private MiniDFSClusterFactory dfsClusterFactory = new MiniDFSClusterFactory();
@@ -87,7 +86,6 @@
         conf = new Job();
         cleanupStores();
         HyracksUtils.init();
-        HyracksUtils.createApp(HYRACKS_APP_NAME);
         FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
         FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
         startHDFS();
@@ -171,7 +169,7 @@
 
         IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
                 HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
-        JobId jobId = client.startJob(HYRACKS_APP_NAME, jobSpec);
+        JobId jobId = client.startJob(jobSpec);
         client.waitForCompletion(jobId);
 
         Assert.assertEquals(true, checkResults());
@@ -203,7 +201,6 @@
 
     @Override
     public void tearDown() throws Exception {
-        HyracksUtils.destroyApp(HYRACKS_APP_NAME);
         HyracksUtils.deinit();
         cleanupHDFS();
     }
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java
index 4b1e11c..442aeae0 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java
@@ -45,17 +45,23 @@
     public void testSchedulerSimple() throws Exception {
         Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
         ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.1").getAddress(), 5099)));
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.2").getAddress(), 5099)));
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.3").getAddress(), 5099)));
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.4").getAddress(), 5099)));
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.5").getAddress(), 5099)));
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.6").getAddress(), 5099)));
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
 
         List<InputSplit> fileSplits = new ArrayList<InputSplit>();
         fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
@@ -68,7 +74,7 @@
         Scheduler scheduler = new Scheduler(ncNameToNcInfos);
         String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
 
-        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc5", "nc6" };
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc6", "nc2", "nc3", "nc5" };
 
         for (int i = 0; i < locationConstraints.length; i++) {
             Assert.assertEquals(locationConstraints[i], expectedResults[i]);
@@ -83,17 +89,23 @@
     public void testSchedulerLargerHDFS() throws Exception {
         Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
         ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.1").getAddress(), 5099)));
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.2").getAddress(), 5099)));
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.3").getAddress(), 5099)));
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.4").getAddress(), 5099)));
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.5").getAddress(), 5099)));
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.6").getAddress(), 5099)));
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
 
         List<InputSplit> fileSplits = new ArrayList<InputSplit>();
         fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
@@ -112,8 +124,8 @@
         Scheduler scheduler = new Scheduler(ncNameToNcInfos);
         String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
 
-        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc3", "nc2", "nc1", "nc4", "nc5", "nc6",
-                "nc6", "nc5" };
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc6", "nc1", "nc4", "nc2", "nc2", "nc3", "nc6", "nc5",
+                "nc3", "nc5" };
 
         for (int i = 0; i < locationConstraints.length; i++) {
             Assert.assertEquals(locationConstraints[i], expectedResults[i]);
@@ -128,17 +140,23 @@
     public void testSchedulerSmallerHDFS() throws Exception {
         Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
         ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.1").getAddress(), 5099)));
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.2").getAddress(), 5099)));
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.3").getAddress(), 5099)));
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.4").getAddress(), 5099)));
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.5").getAddress(), 5099)));
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.6").getAddress(), 5099)));
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
 
         List<InputSplit> fileSplits = new ArrayList<InputSplit>();
         fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
@@ -157,7 +175,7 @@
         Scheduler scheduler = new Scheduler(ncNameToNcInfos);
         String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
 
-        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc3", "nc2", "nc1", "nc4", "nc5", "nc6",
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc5", "nc6",
                 "nc5", "nc6" };
 
         for (int i = 0; i < locationConstraints.length; i++) {
@@ -173,17 +191,23 @@
     public void testSchedulerSmallerHDFSOdd() throws Exception {
         Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
         ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.1").getAddress(), 5099)));
+                .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.2").getAddress(), 5099)));
+                .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.3").getAddress(), 5099)));
+                .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.4").getAddress(), 5099)));
+                .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.5").getAddress(), 5099)));
+                .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+                .getAddress(), 5098)));
         ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
-                .getByName("10.0.0.6").getAddress(), 5099)));
+                .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+                .getAddress(), 5098)));
 
         List<InputSplit> fileSplits = new ArrayList<InputSplit>();
         fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
@@ -203,8 +227,8 @@
         Scheduler scheduler = new Scheduler(ncNameToNcInfos);
         String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
 
-        String[] expectedResults = new String[] { "nc1", "nc3", "nc4", "nc2", "nc3", "nc2", "nc1", "nc4", "nc5", "nc6",
-                "nc5", "nc5", "nc6" };
+        String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc5", "nc1",
+                "nc5", "nc2", "nc4" };
 
         for (int i = 0; i < locationConstraints.length; i++) {
             Assert.assertEquals(locationConstraints[i], expectedResults[i]);
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/topology.xml b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/topology.xml
new file mode 100644
index 0000000..3a0ac7e
--- /dev/null
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/topology.xml
@@ -0,0 +1,32 @@
+<cluster-topology>
+	<network-switch name="all">
+		<network-switch name="rack1">
+			<terminal name="10.0.0.1" />
+			<terminal name="10.0.0.5" />
+			<terminal name="10.0.0.9" />
+			<terminal name="10.0.0.13" />
+			<terminal name="10.0.0.17" />
+		</network-switch>
+		<network-switch name="rack2">
+			<terminal name="10.0.0.2" />
+			<terminal name="10.0.0.6" />
+			<terminal name="10.0.0.10" />
+			<terminal name="10.0.0.14" />
+			<terminal name="10.0.0.18" />
+		</network-switch>
+		<network-switch name="rack3">
+			<terminal name="10.0.0.3" />
+			<terminal name="10.0.0.7" />
+			<terminal name="10.0.0.11" />
+			<terminal name="10.0.0.15" />
+			<terminal name="10.0.0.19" />
+		</network-switch>
+		<network-switch name="rack4">
+			<terminal name="10.0.0.4" />
+			<terminal name="10.0.0.8" />
+			<terminal name="10.0.0.12" />
+			<terminal name="10.0.0.16" />
+			<terminal name="10.0.0.20" />
+		</network-switch>
+	</network-switch>
+</cluster-topology>
\ No newline at end of file
diff --git a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/AbstractHyracksCLIMojo.java b/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/AbstractHyracksCLIMojo.java
deleted file mode 100644
index 7f3faef..0000000
--- a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/AbstractHyracksCLIMojo.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.maven.plugin;
-
-import java.io.File;
-import java.io.PrintWriter;
-
-import org.apache.maven.plugin.MojoExecutionException;
-import org.apache.maven.plugin.MojoFailureException;
-
-public abstract class AbstractHyracksCLIMojo extends AbstractHyracksMojo {
-    private static final String HYRACKS_CLI_SCRIPT = "bin" + File.separator + "hyrackscli";
-
-    /**
-     * @parameter
-     * @required
-     */
-    protected File hyracksCLIHome;
-
-    /**
-     * @parameter
-     * @required
-     */
-    private String ccHost;
-
-    /**
-     * @parameter
-     */
-    private int ccPort;
-
-    @Override
-    public void execute() throws MojoExecutionException, MojoFailureException {
-        StringBuilder buffer = new StringBuilder();
-        buffer.append(createConnectCommand());
-        buffer.append('\n');
-        buffer.append(getCommands());
-        final Process proc = launch(new File(hyracksCLIHome, makeScriptName(HYRACKS_CLI_SCRIPT)), null, null);
-        try {
-            PrintWriter out = new PrintWriter(proc.getOutputStream());
-            out.println(buffer.toString());
-            out.close();
-            proc.waitFor();
-        } catch (Exception e) {
-            throw new MojoExecutionException(e.getMessage());
-        }
-    }
-
-    private String createConnectCommand() {
-        return "connect to \"" + ccHost + (ccPort == 0 ? "" : (":" + ccPort)) + "\";";
-    }
-
-    protected abstract String getCommands();
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksAppDeploymentMojo.java b/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksAppDeploymentMojo.java
deleted file mode 100644
index 76bbb5a..0000000
--- a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksAppDeploymentMojo.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.maven.plugin;
-
-import java.io.File;
-
-/**
- * @goal deploy-app
- */
-public class HyracksAppDeploymentMojo extends AbstractHyracksCLIMojo {
-    /**
-     * @parameter
-     * @required
-     */
-    private String appName;
-
-    /**
-     * @parameter
-     * @required
-     */
-    private File harFile;
-
-    @Override
-    protected String getCommands() {
-        return "create application " + appName + " \"" + harFile.getAbsolutePath() + "\";";
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java b/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java
index 47de024..fc06a68 100644
--- a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java
+++ b/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java
@@ -55,6 +55,7 @@
         cmdLineBuffer.append(" -data-ip-address ").append(dataIpAddress);
         cmdLineBuffer.append(" -node-id ").append(nodeId);
         cmdLineBuffer.append(" -cluster-net-ip-address 127.0.0.1");
+        cmdLineBuffer.append(" -result-ip-address 127.0.0.1");
         if (ccPort != 0) {
             cmdLineBuffer.append(" -cc-port ").append(ccPort);
         }
diff --git a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
index c719bc4..e4df6b9 100644
--- a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
+++ b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
@@ -48,9 +48,9 @@
      * Constructor.
      * 
      * @param localAddress
-     *            - TCP/IP socket address to listen on
+     *            - TCP/IP socket address to listen on. Null for non-listening unidirectional sockets
      * @param listener
-     *            - Callback interface to report channel events
+     *            - Callback interface to report channel events. Null for non-listening unidirectional sockets
      * @param nThreads
      *            - Number of threads to use for data transfer
      * @param maxConnectionAttempts
diff --git a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
index d13a17e..a9061e1 100644
--- a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
+++ b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
@@ -45,15 +45,23 @@
     }
 
     public void start(InetSocketAddress localAddress) throws IOException {
-        serverSocketChannel = ServerSocketChannel.open();
-        ServerSocket serverSocket = serverSocketChannel.socket();
-        serverSocket.bind(localAddress);
-        this.localAddress = (InetSocketAddress) serverSocket.getLocalSocketAddress();
+        // Setup a server socket listening channel only if the TCPEndpoint is a listening endpoint.
+        if (localAddress != null) {
+            serverSocketChannel = ServerSocketChannel.open();
+            ServerSocket serverSocket = serverSocketChannel.socket();
+            serverSocket.bind(localAddress);
+            this.localAddress = (InetSocketAddress) serverSocket.getLocalSocketAddress();
+        }
+
         ioThreads = new IOThread[nThreads];
         for (int i = 0; i < ioThreads.length; ++i) {
             ioThreads[i] = new IOThread();
         }
-        ioThreads[0].registerServerSocket(serverSocketChannel);
+
+        if (localAddress != null) {
+            ioThreads[0].registerServerSocket(serverSocketChannel);
+        }
+
         for (int i = 0; i < ioThreads.length; ++i) {
             ioThreads[i].start();
         }
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
index 0bd872c..0c64d7e 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
@@ -38,11 +38,6 @@
     }
 
     @Override
-    public ClassLoader getClassLoader() {
-        return getClass().getClassLoader();
-    }
-
-    @Override
     public Serializable getDistributedState() {
         return distributedState;
     }
@@ -63,15 +58,9 @@
     }
 
     @Override
-    public String getApplicationName() {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
     public void setMessageBroker(IMessageBroker staticticsConnector) {
         // TODO Auto-generated method stub
-        
+
     }
 
     @Override
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
index c122b25..0ca93b2 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -101,6 +102,11 @@
     }
 
     @Override
+    public IDatasetPartitionManager getDatasetPartitionManager() {
+        return null;
+    }
+
+    @Override
     public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception {
         // TODO Auto-generated method stub
 
diff --git a/hyracks/hyracks-yarn/hyracks-yarn-am/src/main/java/edu/uci/ics/hyracks/yarn/am/HyracksYarnApplicationMaster.java b/hyracks/hyracks-yarn/hyracks-yarn-am/src/main/java/edu/uci/ics/hyracks/yarn/am/HyracksYarnApplicationMaster.java
index a9c4520..e56285a 100644
--- a/hyracks/hyracks-yarn/hyracks-yarn-am/src/main/java/edu/uci/ics/hyracks/yarn/am/HyracksYarnApplicationMaster.java
+++ b/hyracks/hyracks-yarn/hyracks-yarn-am/src/main/java/edu/uci/ics/hyracks/yarn/am/HyracksYarnApplicationMaster.java
@@ -23,14 +23,18 @@
 import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -38,6 +42,7 @@
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.kohsuke.args4j.CmdLineParser;
 
@@ -59,6 +64,10 @@
 
     private final Map<AbstractProcess, AskRecord> proc2AskMap;
 
+    private final AtomicInteger lastResponseId;
+
+    private final ApplicationAttemptId appAttemptId;
+
     private YarnConfiguration config;
 
     private AMRMConnection amrmc;
@@ -73,6 +82,11 @@
         asks = new ArrayList<ResourceRequest>();
         resource2AskMap = new HashMap<Resource, Set<AskRecord>>();
         proc2AskMap = new HashMap<AbstractProcess, AskRecord>();
+        lastResponseId = new AtomicInteger();
+
+        String containerIdStr = System.getenv(ApplicationConstants.AM_CONTAINER_ID_ENV);
+        ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+        appAttemptId = containerId.getApplicationAttemptId();
     }
 
     private void run() throws Exception {
@@ -103,7 +117,7 @@
         rsrcRequest.setHostName(cSpec.getHostname());
 
         Priority pri = Records.newRecord(Priority.class);
-        pri.setPriority(100);
+        pri.setPriority(0);
         rsrcRequest.setPriority(pri);
 
         Resource capability = Records.newRecord(Resource.class);
@@ -124,7 +138,8 @@
         arSet.add(ar);
         proc2AskMap.put(proc, ar);
 
-        System.err.println(proc + " -> " + rsrcRequest);
+        System.err.println(proc + " -> [" + rsrcRequest.getHostName() + ", " + rsrcRequest.getNumContainers() + ", "
+                + rsrcRequest.getPriority() + ", " + rsrcRequest.getCapability().getMemory() + "]");
 
         asks.add(rsrcRequest);
     }
@@ -162,11 +177,14 @@
 
     private synchronized void populateAllocateRequest(AllocateRequest hb) {
         hb.addAllAsks(asks);
-        asks.clear();
+        hb.addAllReleases(new ArrayList<ContainerId>());
+        hb.setResponseId(lastResponseId.incrementAndGet());
+        hb.setApplicationAttemptId(appAttemptId);
     }
 
     private synchronized void processAllocation(List<Container> allocatedContainers,
             List<ContainerStatus> completedContainers) {
+        System.err.println(allocatedContainers);
         for (Container c : allocatedContainers) {
             System.err.println("Got container: " + c.getContainerStatus());
             NodeId nodeId = c.getNodeId();
diff --git a/hyracks/hyracks-yarn/hyracks-yarn-client/src/main/java/edu/uci/ics/hyracks/yarn/client/LaunchHyracksApplication.java b/hyracks/hyracks-yarn/hyracks-yarn-client/src/main/java/edu/uci/ics/hyracks/yarn/client/LaunchHyracksApplication.java
index e12891e..0eb930e 100644
--- a/hyracks/hyracks-yarn/hyracks-yarn-client/src/main/java/edu/uci/ics/hyracks/yarn/client/LaunchHyracksApplication.java
+++ b/hyracks/hyracks-yarn/hyracks-yarn-client/src/main/java/edu/uci/ics/hyracks/yarn/client/LaunchHyracksApplication.java
@@ -53,7 +53,6 @@
 
         File amZipFile = new File(System.getProperty("basedir") + "/hyracks-yarn-am/hyracks-yarn-am.zip");
         localResources.put("archive", LocalResourceHelper.createArchiveResource(conf, amZipFile));
-        localResources.put("manifest.xml", LocalResourceHelper.createFileResource(conf, options.hcManifest));
         clCtx.setLocalResources(localResources);
 
         String command = "./archive/bin/hyracks-yarn-am 1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout"
@@ -84,10 +83,19 @@
         @Option(name = "-application-name", required = true, usage = "Application Name")
         String appName;
 
+        @Option(name = "-am-host", required = false, usage = "Application master host name (default: *). Currently has NO effect")
+        String amHostName = "*";
+
         @Option(name = "-am-memory", required = false, usage = "Application Master memory requirements")
         int amMemory = 128;
 
-        @Option(name = "-hyracks-cluster-manifest", required = true, usage = "Hyracks Cluster Manifest file")
-        File hcManifest;
+        @Option(name = "-workers", required = true, usage = "Number of worker containers")
+        int nWorkers;
+
+        @Option(name = "-worker-memory", required = true, usage = "Amount of memory to provide to each worker")
+        int workerMemory;
+
+        @Option(name = "-extra-jars", required = false, usage = "Other jars that need to be added to the classpath")
+        String extraJars = "";
     }
 }
\ No newline at end of file
diff --git a/hyracks/pom.xml b/hyracks/pom.xml
index 0de6790..1592384 100644
--- a/hyracks/pom.xml
+++ b/hyracks/pom.xml
@@ -29,7 +29,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
-        <version>2.13</version>
+        <version>2.12</version>
         <configuration>
           <forkMode>pertest</forkMode>
           <argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties -Xdebug -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${jvm.extraargs} -Xmx2048m</argLine>
@@ -38,16 +38,6 @@
     </plugins>
   </build>
 
-  <reporting>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-changelog-plugin</artifactId>
-        <version>2.2</version>
-      </plugin>
-    </plugins>
-  </reporting>
-
   <distributionManagement>
     <repository>
       <id>hyracks-releases</id>
@@ -83,13 +73,14 @@
   <modules>
     <module>hyracks-ipc</module>
     <module>hyracks-api</module>
+    <module>hyracks-comm</module>
+    <module>hyracks-client</module>
     <module>hyracks-dataflow-common</module>
     <module>hyracks-dataflow-std</module>
     <module>hyracks-dataflow-hadoop</module>
     <module>hyracks-control</module>
     <module>hyracks-net</module>
     <module>hyracks-data</module>
-    <module>hyracks-cli</module>
     <module>hyracks-storage-common</module>
     <module>hyracks-storage-am-common</module>
     <module>hyracks-storage-am-bloomfilter</module>
diff --git a/pregelix/pom.xml b/pregelix/pom.xml
index 7d08fb7..6e28021 100644
--- a/pregelix/pom.xml
+++ b/pregelix/pom.xml
@@ -44,7 +44,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
-        <version>2.13</version>
+        <version>2.12</version>
         <configuration>
             <forkMode>pertest</forkMode>
             <argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties -Xdebug -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${jvm.extraargs}</argLine>
@@ -53,16 +53,6 @@
     </plugins>
   </build>
 
-  <reporting>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-changelog-plugin</artifactId>
-        <version>2.2</version>
-      </plugin>
-    </plugins>
-  </reporting>
-
   <distributionManagement>
     <repository>
       <id>hyracks-releases</id>
diff --git a/pregelix/pregelix-api/pom.xml b/pregelix/pregelix-api/pom.xml
index 8212e1c..10efa59 100644
--- a/pregelix/pregelix-api/pom.xml
+++ b/pregelix/pregelix-api/pom.xml
@@ -41,9 +41,10 @@
 				</configuration>
 			</plugin>
 			<plugin>
-				<artifactId>maven-clean-plugin</artifactId>
-				<version>2.5</version>
-				<configuration>
+				<groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-clean-plugin</artifactId>
+                <version>2.4.1</version>
+                <configuration>
 					<filesets>
 						<fileset>
 							<directory>.</directory>
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
index 3a98fd9..cd49184 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
@@ -49,510 +49,525 @@
  */
 @SuppressWarnings("rawtypes")
 public abstract class Vertex<I extends WritableComparable, V extends Writable, E extends Writable, M extends Writable>
-        implements Writable {
-    private static long superstep = 0;
-    /** Class-wide number of vertices */
-    private static long numVertices = -1;
-    /** Class-wide number of edges */
-    private static long numEdges = -1;
-    /** Vertex id */
-    private I vertexId = null;
-    /** Vertex value */
-    private V vertexValue = null;
-    /** Map of destination vertices and their edge values */
-    private final List<Edge<I, E>> destEdgeList = new ArrayList<Edge<I, E>>();
-    /** If true, do not do anymore computation on this vertex. */
-    boolean halt = false;
-    /** List of incoming messages from the previous superstep */
-    private final List<M> msgList = new ArrayList<M>();
-    /** map context */
-    private static TaskAttemptContext context = null;
-    /** a delegate for hyracks stuff */
-    private VertexDelegate<I, V, E, M> delegate = new VertexDelegate<I, V, E, M>(this);
-    /** this vertex is updated or not */
-    private boolean updated = false;
-    /** has outgoing messages */
-    private boolean hasMessage = false;
-    /** created new vertex */
-    private boolean createdNewLiveVertex = false;
+		implements Writable {
+	private static long superstep = 0;
+	/** Class-wide number of vertices */
+	private static long numVertices = -1;
+	/** Class-wide number of edges */
+	private static long numEdges = -1;
+	/** Vertex id */
+	private I vertexId = null;
+	/** Vertex value */
+	private V vertexValue = null;
+	/** Map of destination vertices and their edge values */
+	private final List<Edge<I, E>> destEdgeList = new ArrayList<Edge<I, E>>();
+	/** If true, do not do anymore computation on this vertex. */
+	boolean halt = false;
+	/** List of incoming messages from the previous superstep */
+	private final List<M> msgList = new ArrayList<M>();
+	/** map context */
+	private static TaskAttemptContext context = null;
+	/** a delegate for hyracks stuff */
+	private VertexDelegate<I, V, E, M> delegate = new VertexDelegate<I, V, E, M>(
+			this);
+	/** this vertex is updated or not */
+	private boolean updated = false;
+	/** has outgoing messages */
+	private boolean hasMessage = false;
+	/** created new vertex */
+	private boolean createdNewLiveVertex = false;
 
-    /**
-     * use object pool for re-using objects
-     */
-    private List<Edge<I, E>> edgePool = new ArrayList<Edge<I, E>>();
-    private List<M> msgPool = new ArrayList<M>();
-    private List<V> valuePool = new ArrayList<V>();
-    private int usedEdge = 0;
-    private int usedMessage = 0;
-    private int usedValue = 0;
+	/**
+	 * use object pool for re-using objects
+	 */
+	private List<Edge<I, E>> edgePool = new ArrayList<Edge<I, E>>();
+	private List<M> msgPool = new ArrayList<M>();
+	private List<V> valuePool = new ArrayList<V>();
+	private int usedEdge = 0;
+	private int usedMessage = 0;
+	private int usedValue = 0;
 
-    /**
-     * The key method that users need to implement
-     * 
-     * @param msgIterator
-     *            an iterator of incoming messages
-     */
-    public abstract void compute(Iterator<M> msgIterator);
+	/**
+	 * The key method that users need to implement
+	 * 
+	 * @param msgIterator
+	 *            an iterator of incoming messages
+	 */
+	public abstract void compute(Iterator<M> msgIterator);
 
-    /**
-     * Add an edge for the vertex.
-     * 
-     * @param targetVertexId
-     * @param edgeValue
-     * @return successful or not
-     */
-    public final boolean addEdge(I targetVertexId, E edgeValue) {
-        Edge<I, E> edge = this.allocateEdge();
-        edge.setDestVertexId(targetVertexId);
-        edge.setEdgeValue(edgeValue);
-        destEdgeList.add(edge);
-        return true;
-    }
+	/**
+	 * Add an edge for the vertex.
+	 * 
+	 * @param targetVertexId
+	 * @param edgeValue
+	 * @return successful or not
+	 */
+	public final boolean addEdge(I targetVertexId, E edgeValue) {
+		Edge<I, E> edge = this.allocateEdge();
+		edge.setDestVertexId(targetVertexId);
+		edge.setEdgeValue(edgeValue);
+		destEdgeList.add(edge);
+		updated = true;
+		return true;
+	}
 
-    /**
-     * Initialize a new vertex
-     * 
-     * @param vertexId
-     * @param vertexValue
-     * @param edges
-     * @param messages
-     */
-    public void initialize(I vertexId, V vertexValue, Map<I, E> edges, List<M> messages) {
-        if (vertexId != null) {
-            setVertexId(vertexId);
-        }
-        if (vertexValue != null) {
-            setVertexValue(vertexValue);
-        }
-        destEdgeList.clear();
-        if (edges != null && !edges.isEmpty()) {
-            for (Map.Entry<I, E> entry : edges.entrySet()) {
-                destEdgeList.add(new Edge<I, E>(entry.getKey(), entry.getValue()));
-            }
-        }
-        if (messages != null && !messages.isEmpty()) {
-            msgList.addAll(messages);
-        }
-    }
+	/**
+	 * Initialize a new vertex
+	 * 
+	 * @param vertexId
+	 * @param vertexValue
+	 * @param edges
+	 * @param messages
+	 */
+	public void initialize(I vertexId, V vertexValue, Map<I, E> edges,
+			List<M> messages) {
+		if (vertexId != null) {
+			setVertexId(vertexId);
+		}
+		if (vertexValue != null) {
+			setVertexValue(vertexValue);
+		}
+		destEdgeList.clear();
+		if (edges != null && !edges.isEmpty()) {
+			for (Map.Entry<I, E> entry : edges.entrySet()) {
+				destEdgeList.add(new Edge<I, E>(entry.getKey(), entry
+						.getValue()));
+			}
+		}
+		if (messages != null && !messages.isEmpty()) {
+			msgList.addAll(messages);
+		}
+	}
 
-    /**
-     * reset a vertex object: clear its internal states
-     */
-    public void reset() {
-        usedEdge = 0;
-        usedMessage = 0;
-        usedValue = 0;
-    }
+	/**
+	 * reset a vertex object: clear its internal states
+	 */
+	public void reset() {
+		usedEdge = 0;
+		usedMessage = 0;
+		usedValue = 0;
+		updated = false;
+	}
 
-    /**
-     * Set the vertex id
-     * 
-     * @param vertexId
-     */
-    public final void setVertexId(I vertexId) {
-        this.vertexId = vertexId;
-        delegate.setVertexId(vertexId);
-    }
+	/**
+	 * Set the vertex id
+	 * 
+	 * @param vertexId
+	 */
+	public final void setVertexId(I vertexId) {
+		this.vertexId = vertexId;
+		delegate.setVertexId(vertexId);
+	}
 
-    /**
-     * Get the vertex id
-     * 
-     * @return vertex id
-     */
-    public final I getVertexId() {
-        return vertexId;
-    }
+	/**
+	 * Get the vertex id
+	 * 
+	 * @return vertex id
+	 */
+	public final I getVertexId() {
+		return vertexId;
+	}
 
-    /**
-     * Get the vertex value
-     * 
-     * @return the vertex value
-     */
-    public final V getVertexValue() {
-        return vertexValue;
-    }
+	/**
+	 * Get the vertex value
+	 * 
+	 * @return the vertex value
+	 */
+	public final V getVertexValue() {
+		return vertexValue;
+	}
 
-    /**
-     * Set the vertex value
-     * 
-     * @param vertexValue
-     */
-    public final void setVertexValue(V vertexValue) {
-        this.vertexValue = vertexValue;
-        this.updated = true;
-    }
+	/**
+	 * Set the vertex value
+	 * 
+	 * @param vertexValue
+	 */
+	public final void setVertexValue(V vertexValue) {
+		this.vertexValue = vertexValue;
+		this.updated = true;
+	}
 
-    /***
-     * Send a message to a specific vertex
-     * 
-     * @param id
-     *            the receiver vertex id
-     * @param msg
-     *            the message
-     */
-    public final void sendMsg(I id, M msg) {
-        if (msg == null) {
-            throw new IllegalArgumentException("sendMsg: Cannot send null message to " + id);
-        }
-        delegate.sendMsg(id, msg);
-        this.hasMessage = true;
-    }
+	/***
+	 * Send a message to a specific vertex
+	 * 
+	 * @param id
+	 *            the receiver vertex id
+	 * @param msg
+	 *            the message
+	 */
+	public final void sendMsg(I id, M msg) {
+		if (msg == null) {
+			throw new IllegalArgumentException(
+					"sendMsg: Cannot send null message to " + id);
+		}
+		delegate.sendMsg(id, msg);
+		this.hasMessage = true;
+	}
 
-    /**
-     * Send a message to all direct outgoing neighbors
-     * 
-     * @param msg
-     *            the message
-     */
-    public final void sendMsgToAllEdges(M msg) {
-        if (msg == null) {
-            throw new IllegalArgumentException("sendMsgToAllEdges: Cannot send null message to all edges");
-        }
-        for (Edge<I, E> edge : destEdgeList) {
-            sendMsg(edge.getDestVertexId(), msg);
-        }
-    }
+	/**
+	 * Send a message to all direct outgoing neighbors
+	 * 
+	 * @param msg
+	 *            the message
+	 */
+	public final void sendMsgToAllEdges(M msg) {
+		if (msg == null) {
+			throw new IllegalArgumentException(
+					"sendMsgToAllEdges: Cannot send null message to all edges");
+		}
+		for (Edge<I, E> edge : destEdgeList) {
+			sendMsg(edge.getDestVertexId(), msg);
+		}
+	}
 
-    /**
-     * Vote to halt. Once all vertex vote to halt and no more messages, a
-     * Pregelix job will terminate.
-     */
-    public final void voteToHalt() {
-        halt = true;
-    }
+	/**
+	 * Vote to halt. Once all vertex vote to halt and no more messages, a
+	 * Pregelix job will terminate.
+	 */
+	public final void voteToHalt() {
+		halt = true;
+		updated = true;
+	}
 
-    /**
-     * @return the vertex is halted (true) or not (false)
-     */
-    public final boolean isHalted() {
-        return halt;
-    }
+	/**
+	 * @return the vertex is halted (true) or not (false)
+	 */
+	public final boolean isHalted() {
+		return halt;
+	}
 
-    @Override
-    final public void readFields(DataInput in) throws IOException {
-        reset();
-        if (vertexId == null)
-            vertexId = BspUtils.<I> createVertexIndex(getContext().getConfiguration());
-        vertexId.readFields(in);
-        delegate.setVertexId(vertexId);
-        boolean hasVertexValue = in.readBoolean();
+	@Override
+	final public void readFields(DataInput in) throws IOException {
+		reset();
+		if (vertexId == null)
+			vertexId = BspUtils.<I> createVertexIndex(getContext()
+					.getConfiguration());
+		vertexId.readFields(in);
+		delegate.setVertexId(vertexId);
+		boolean hasVertexValue = in.readBoolean();
 
-        if (hasVertexValue) {
-            vertexValue = allocateValue();
-            vertexValue.readFields(in);
-            delegate.setVertex(this);
-        }
-        destEdgeList.clear();
-        long edgeMapSize = SerDeUtils.readVLong(in);
-        for (long i = 0; i < edgeMapSize; ++i) {
-            Edge<I, E> edge = allocateEdge();
-            edge.setConf(getContext().getConfiguration());
-            edge.readFields(in);
-            addEdge(edge);
-        }
-        msgList.clear();
-        long msgListSize = SerDeUtils.readVLong(in);
-        for (long i = 0; i < msgListSize; ++i) {
-            M msg = allocateMessage();
-            msg.readFields(in);
-            msgList.add(msg);
-        }
-        halt = in.readBoolean();
-        updated = false;
-        hasMessage = false;
-        createdNewLiveVertex = false;
-    }
+		if (hasVertexValue) {
+			vertexValue = allocateValue();
+			vertexValue.readFields(in);
+			delegate.setVertex(this);
+		}
+		destEdgeList.clear();
+		long edgeMapSize = SerDeUtils.readVLong(in);
+		for (long i = 0; i < edgeMapSize; ++i) {
+			Edge<I, E> edge = allocateEdge();
+			edge.setConf(getContext().getConfiguration());
+			edge.readFields(in);
+			addEdge(edge);
+		}
+		msgList.clear();
+		long msgListSize = SerDeUtils.readVLong(in);
+		for (long i = 0; i < msgListSize; ++i) {
+			M msg = allocateMessage();
+			msg.readFields(in);
+			msgList.add(msg);
+		}
+		halt = in.readBoolean();
+		updated = false;
+		hasMessage = false;
+		createdNewLiveVertex = false;
+	}
 
-    @Override
-    public void write(DataOutput out) throws IOException {
-        vertexId.write(out);
-        out.writeBoolean(vertexValue != null);
-        if (vertexValue != null) {
-            vertexValue.write(out);
-        }
-        SerDeUtils.writeVLong(out, destEdgeList.size());
-        for (Edge<I, E> edge : destEdgeList) {
-            edge.write(out);
-        }
-        SerDeUtils.writeVLong(out, msgList.size());
-        for (M msg : msgList) {
-            msg.write(out);
-        }
-        out.writeBoolean(halt);
-    }
+	@Override
+	public void write(DataOutput out) throws IOException {
+		vertexId.write(out);
+		out.writeBoolean(vertexValue != null);
+		if (vertexValue != null) {
+			vertexValue.write(out);
+		}
+		SerDeUtils.writeVLong(out, destEdgeList.size());
+		for (Edge<I, E> edge : destEdgeList) {
+			edge.write(out);
+		}
+		SerDeUtils.writeVLong(out, msgList.size());
+		for (M msg : msgList) {
+			msg.write(out);
+		}
+		out.writeBoolean(halt);
+	}
 
-    /**
-     * Get the list of incoming messages
-     * 
-     * @return the list of messages
-     */
-    public List<M> getMsgList() {
-        return msgList;
-    }
+	/**
+	 * Get the list of incoming messages
+	 * 
+	 * @return the list of messages
+	 */
+	public List<M> getMsgList() {
+		return msgList;
+	}
 
-    /**
-     * Get outgoing edge list
-     * 
-     * @return a list of outgoing edges
-     */
-    public List<Edge<I, E>> getEdges() {
-        return this.destEdgeList;
-    }
+	/**
+	 * Get outgoing edge list
+	 * 
+	 * @return a list of outgoing edges
+	 */
+	public List<Edge<I, E>> getEdges() {
+		return this.destEdgeList;
+	}
 
-    @Override
-    @SuppressWarnings("unchecked")
-    public String toString() {
-        Collections.sort(destEdgeList);
-        StringBuffer edgeBuffer = new StringBuffer();
-        edgeBuffer.append("(");
-        for (Edge<I, E> edge : destEdgeList) {
-            edgeBuffer.append(edge.getDestVertexId()).append(",");
-        }
-        edgeBuffer.append(")");
-        return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue() + ", edges=" + edgeBuffer + ")";
-    }
+	@Override
+	@SuppressWarnings("unchecked")
+	public String toString() {
+		Collections.sort(destEdgeList);
+		StringBuffer edgeBuffer = new StringBuffer();
+		edgeBuffer.append("(");
+		for (Edge<I, E> edge : destEdgeList) {
+			edgeBuffer.append(edge.getDestVertexId()).append(",");
+		}
+		edgeBuffer.append(")");
+		return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue()
+				+ ", edges=" + edgeBuffer + ")";
+	}
 
-    /**
-     * Get the number of outgoing edges
-     * 
-     * @return the number of outging edges
-     */
-    public int getNumOutEdges() {
-        return destEdgeList.size();
-    }
+	/**
+	 * Get the number of outgoing edges
+	 * 
+	 * @return the number of outging edges
+	 */
+	public int getNumOutEdges() {
+		return destEdgeList.size();
+	}
 
-    /**
-     * Pregelix internal use only
-     * 
-     * @param writers
-     */
-    public void setOutputWriters(List<IFrameWriter> writers) {
-        delegate.setOutputWriters(writers);
-    }
+	/**
+	 * Pregelix internal use only
+	 * 
+	 * @param writers
+	 */
+	public void setOutputWriters(List<IFrameWriter> writers) {
+		delegate.setOutputWriters(writers);
+	}
 
-    /**
-     * Pregelix internal use only
-     * 
-     * @param writers
-     */
-    public void setOutputAppenders(List<FrameTupleAppender> appenders) {
-        delegate.setOutputAppenders(appenders);
-    }
+	/**
+	 * Pregelix internal use only
+	 * 
+	 * @param writers
+	 */
+	public void setOutputAppenders(List<FrameTupleAppender> appenders) {
+		delegate.setOutputAppenders(appenders);
+	}
 
-    /**
-     * Pregelix internal use only
-     * 
-     * @param writers
-     */
-    public void setOutputTupleBuilders(List<ArrayTupleBuilder> tbs) {
-        delegate.setOutputTupleBuilders(tbs);
-    }
+	/**
+	 * Pregelix internal use only
+	 * 
+	 * @param writers
+	 */
+	public void setOutputTupleBuilders(List<ArrayTupleBuilder> tbs) {
+		delegate.setOutputTupleBuilders(tbs);
+	}
 
-    /**
-     * Pregelix internal use only
-     * 
-     * @param writers
-     */
-    public void finishCompute() throws IOException {
-        delegate.finishCompute();
-    }
+	/**
+	 * Pregelix internal use only
+	 * 
+	 * @param writers
+	 */
+	public void finishCompute() throws IOException {
+		delegate.finishCompute();
+	}
 
-    /**
-     * Pregelix internal use only
-     */
-    public boolean hasUpdate() {
-        return this.updated;
-    }
+	/**
+	 * Pregelix internal use only
+	 */
+	public boolean hasUpdate() {
+		return this.updated;
+	}
 
-    /**
-     * Pregelix internal use only
-     */
-    public boolean hasMessage() {
-        return this.hasMessage;
-    }
+	/**
+	 * Pregelix internal use only
+	 */
+	public boolean hasMessage() {
+		return this.hasMessage;
+	}
 
-    /**
-     * Pregelix internal use only
-     */
-    public boolean createdNewLiveVertex() {
-        return this.createdNewLiveVertex;
-    }
+	/**
+	 * Pregelix internal use only
+	 */
+	public boolean createdNewLiveVertex() {
+		return this.createdNewLiveVertex;
+	}
 
-    /**
-     * sort the edges
-     */
-    @SuppressWarnings("unchecked")
-    public void sortEdges() {
-        Collections.sort(destEdgeList);
-    }
+	/**
+	 * sort the edges
+	 */
+	@SuppressWarnings("unchecked")
+	public void sortEdges() {
+		updated = true;
+		Collections.sort(destEdgeList);
+	}
 
-    /**
-     * Allocate a new edge from the edge pool
-     */
-    private Edge<I, E> allocateEdge() {
-        Edge<I, E> edge;
-        if (usedEdge < edgePool.size()) {
-            edge = edgePool.get(usedEdge);
-            usedEdge++;
-        } else {
-            edge = new Edge<I, E>();
-            edgePool.add(edge);
-            usedEdge++;
-        }
-        return edge;
-    }
+	/**
+	 * Allocate a new edge from the edge pool
+	 */
+	private Edge<I, E> allocateEdge() {
+		Edge<I, E> edge;
+		if (usedEdge < edgePool.size()) {
+			edge = edgePool.get(usedEdge);
+			usedEdge++;
+		} else {
+			edge = new Edge<I, E>();
+			edgePool.add(edge);
+			usedEdge++;
+		}
+		return edge;
+	}
 
-    /**
-     * Allocate a new message from the message pool
-     */
-    private M allocateMessage() {
-        M message;
-        if (usedMessage < msgPool.size()) {
-            message = msgPool.get(usedEdge);
-            usedMessage++;
-        } else {
-            message = BspUtils.<M> createMessageValue(getContext().getConfiguration());
-            msgPool.add(message);
-            usedMessage++;
-        }
-        return message;
-    }
+	/**
+	 * Allocate a new message from the message pool
+	 */
+	private M allocateMessage() {
+		M message;
+		if (usedMessage < msgPool.size()) {
+			message = msgPool.get(usedEdge);
+			usedMessage++;
+		} else {
+			message = BspUtils.<M> createMessageValue(getContext()
+					.getConfiguration());
+			msgPool.add(message);
+			usedMessage++;
+		}
+		return message;
+	}
 
-    /**
-     * Set the global superstep for all the vertices (internal use)
-     * 
-     * @param superstep
-     *            New superstep
-     */
-    public static final void setSuperstep(long superstep) {
-        Vertex.superstep = superstep;
-    }
+	/**
+	 * Set the global superstep for all the vertices (internal use)
+	 * 
+	 * @param superstep
+	 *            New superstep
+	 */
+	public static final void setSuperstep(long superstep) {
+		Vertex.superstep = superstep;
+	}
 
-    /**
-     * Add an outgoing edge into the vertex
-     * 
-     * @param edge
-     *            the edge to be added
-     * @return true if the edge list changed as a result of this call
-     */
-    public boolean addEdge(Edge<I, E> edge) {
-        edge.setConf(getContext().getConfiguration());
-        return destEdgeList.add(edge);
-    }
+	/**
+	 * Add an outgoing edge into the vertex
+	 * 
+	 * @param edge
+	 *            the edge to be added
+	 * @return true if the edge list changed as a result of this call
+	 */
+	public boolean addEdge(Edge<I, E> edge) {
+		edge.setConf(getContext().getConfiguration());
+		updated = true;
+		return destEdgeList.add(edge);
+	}
 
-    /**
-     * remove an outgoing edge in the graph
-     * 
-     * @param edge
-     *            the edge to be removed
-     * @return true if the edge is in the edge list of the vertex
-     */
-    public boolean removeEdge(Edge<I, E> edge) {
-        return destEdgeList.remove(edge);
-    }
+	/**
+	 * remove an outgoing edge in the graph
+	 * 
+	 * @param edge
+	 *            the edge to be removed
+	 * @return true if the edge is in the edge list of the vertex
+	 */
+	public boolean removeEdge(Edge<I, E> edge) {
+		updated = true;
+		return destEdgeList.remove(edge);
+	}
 
-    /**
-     * Add a new vertex into the graph
-     * 
-     * @param vertexId
-     *            the vertex id
-     * @param vertex
-     *            the vertex
-     */
-    public final void addVertex(I vertexId, Vertex vertex) {
-        createdNewLiveVertex |= !vertex.isHalted();
-        delegate.addVertex(vertexId, vertex);
-    }
+	/**
+	 * Add a new vertex into the graph
+	 * 
+	 * @param vertexId
+	 *            the vertex id
+	 * @param vertex
+	 *            the vertex
+	 */
+	public final void addVertex(I vertexId, Vertex vertex) {
+		createdNewLiveVertex |= !vertex.isHalted();
+		delegate.addVertex(vertexId, vertex);
+	}
 
-    /**
-     * Delete a vertex from id
-     * 
-     * @param vertexId
-     *            the vertex id
-     */
-    public final void deleteVertex(I vertexId) {
-        delegate.deleteVertex(vertexId);
-    }
+	/**
+	 * Delete a vertex from id
+	 * 
+	 * @param vertexId
+	 *            the vertex id
+	 */
+	public final void deleteVertex(I vertexId) {
+		delegate.deleteVertex(vertexId);
+	}
 
-    /**
-     * Allocate a vertex value from the object pool
-     * 
-     * @return a vertex value instance
-     */
-    private V allocateValue() {
-        V value;
-        if (usedValue < valuePool.size()) {
-            value = valuePool.get(usedValue);
-            usedValue++;
-        } else {
-            value = BspUtils.<V> createVertexValue(getContext().getConfiguration());
-            valuePool.add(value);
-            usedValue++;
-        }
-        return value;
-    }
+	/**
+	 * Allocate a vertex value from the object pool
+	 * 
+	 * @return a vertex value instance
+	 */
+	private V allocateValue() {
+		V value;
+		if (usedValue < valuePool.size()) {
+			value = valuePool.get(usedValue);
+			usedValue++;
+		} else {
+			value = BspUtils.<V> createVertexValue(getContext()
+					.getConfiguration());
+			valuePool.add(value);
+			usedValue++;
+		}
+		return value;
+	}
 
-    /**
-     * Get the current global superstep number
-     * 
-     * @return the current superstep number
-     */
-    public static final long getSuperstep() {
-        return superstep;
-    }
+	/**
+	 * Get the current global superstep number
+	 * 
+	 * @return the current superstep number
+	 */
+	public static final long getSuperstep() {
+		return superstep;
+	}
 
-    /**
-     * Set the total number of vertices from the last superstep.
-     * 
-     * @param numVertices
-     *            Aggregate vertices in the last superstep
-     */
-    public static final void setNumVertices(long numVertices) {
-        Vertex.numVertices = numVertices;
-    }
+	/**
+	 * Set the total number of vertices from the last superstep.
+	 * 
+	 * @param numVertices
+	 *            Aggregate vertices in the last superstep
+	 */
+	public static final void setNumVertices(long numVertices) {
+		Vertex.numVertices = numVertices;
+	}
 
-    /**
-     * Get the number of vertexes in the graph
-     * 
-     * @return the number of vertexes in the graph
-     */
-    public static final long getNumVertices() {
-        return numVertices;
-    }
+	/**
+	 * Get the number of vertexes in the graph
+	 * 
+	 * @return the number of vertexes in the graph
+	 */
+	public static final long getNumVertices() {
+		return numVertices;
+	}
 
-    /**
-     * Set the total number of edges from the last superstep.
-     * 
-     * @param numEdges
-     *            Aggregate edges in the last superstep
-     */
-    public static void setNumEdges(long numEdges) {
-        Vertex.numEdges = numEdges;
-    }
+	/**
+	 * Set the total number of edges from the last superstep.
+	 * 
+	 * @param numEdges
+	 *            Aggregate edges in the last superstep
+	 */
+	public static void setNumEdges(long numEdges) {
+		Vertex.numEdges = numEdges;
+	}
 
-    /**
-     * Get the number of edges from this graph
-     * 
-     * @return the number of edges in the graph
-     */
-    public static final long getNumEdges() {
-        return numEdges;
-    }
+	/**
+	 * Get the number of edges from this graph
+	 * 
+	 * @return the number of edges in the graph
+	 */
+	public static final long getNumEdges() {
+		return numEdges;
+	}
 
-    /**
-     * Pregelix internal use only
-     */
-    public static final TaskAttemptContext getContext() {
-        return context;
-    }
+	/**
+	 * Pregelix internal use only
+	 */
+	public static final TaskAttemptContext getContext() {
+		return context;
+	}
 
-    /**
-     * Pregelix internal use only
-     * 
-     * @param context
-     */
-    public static final void setContext(TaskAttemptContext context) {
-        Vertex.context = context;
-    }
+	/**
+	 * Pregelix internal use only
+	 * 
+	 * @param context
+	 */
+	public static final void setContext(TaskAttemptContext context) {
+		Vertex.context = context;
+	}
 
 }
diff --git a/pregelix/pregelix-core/pom.xml b/pregelix/pregelix-core/pom.xml
index 4c63452..f084614 100644
--- a/pregelix/pregelix-core/pom.xml
+++ b/pregelix/pregelix-core/pom.xml
@@ -19,8 +19,9 @@
 	<build>
 		<plugins>
 			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-jar-plugin</artifactId>
-				<version>2.4</version>
+				<version>2.3.2</version>
 				<executions>
 					<execution>
 						<id>balancer</id>
@@ -73,7 +74,7 @@
 			<plugin>
 				<groupId>org.codehaus.mojo</groupId>
 				<artifactId>appassembler-maven-plugin</artifactId>
-				<version>1.3</version>
+                <version>1.3</version>
 				<executions>
 					<execution>
 						<configuration>
@@ -165,7 +166,7 @@
 			</plugin>
 			<plugin>
 				<artifactId>maven-clean-plugin</artifactId>
-				<version>2.5</version>
+                <version>2.4.1</version>
 				<configuration>
 					<filesets>
 						<fileset>
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
index f07a246..d9b267d 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
@@ -141,22 +141,11 @@
             start = System.currentTimeMillis();
             runHDFSWRite(jobGen);
             runCleanup(jobGen);
-            destroyApplication(applicationName);
             end = System.currentTimeMillis();
             time = end - start;
             LOG.info("result writing finished " + time + "ms");
             LOG.info("job finished");
         } catch (Exception e) {
-            try {
-                /**
-                 * destroy application if there is any exception
-                 */
-                if (hcc != null) {
-                    destroyApplication(applicationName);
-                }
-            } catch (Exception e2) {
-                throw new HyracksException(e2);
-            }
             throw new HyracksException(e);
         }
     }
@@ -214,8 +203,8 @@
 
     private void execute(JobSpecification job) throws Exception {
         job.setUseConnectorPolicyForScheduling(false);
-        JobId jobId = hcc.startJob(applicationName, job,
-                profiling ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
+        JobId jobId = hcc
+                .startJob(job, profiling ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
         hcc.waitForCompletion(jobId);
     }
 
@@ -230,15 +219,11 @@
         LOG.info("jar packing finished " + (end - start) + "ms");
 
         start = System.currentTimeMillis();
-        hcc.createApplication(applicationName, appZip);
+        // TODO: Fix this step to use Yarn
+        //hcc.createApplication(applicationName, appZip);
         end = System.currentTimeMillis();
         LOG.info("jar deployment finished " + (end - start) + "ms");
     }
-
-    public void destroyApplication(String appName) throws Exception {
-        hcc.destroyApplication(appName);
-    }
-
 }
 
 class FileFilter implements FilenameFilter {
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
index 419e4b0..fe2fcac 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
@@ -262,12 +262,10 @@
          */
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 3, insertOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 4,
-                deleteOp, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 4, deleteOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
 
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 5,
-                btreeBulkLoad, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 5, btreeBulkLoad, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
                 localGby, 0, globalGby, 0);
@@ -487,11 +485,9 @@
         /**
          * connect the insert/delete operator
          */
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 3,
-                insertOp, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 3, insertOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 4,
-                deleteOp, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 4, deleteOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
 
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 5, btreeBulkLoad, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
index 27394e6..f1eceb7 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
@@ -241,11 +241,9 @@
         /**
          * connect the insert/delete operator
          */
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 3, insertOp,
-                0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 3, insertOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 4, deleteOp,
-                0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 4, deleteOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
 
         /**
@@ -449,9 +447,9 @@
         /**
          * connect the insert/delete operator
          */
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 3, insertOp, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 3, insertOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 4, deleteOp, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 4, deleteOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
index 91a629c..314c393 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
@@ -219,26 +219,24 @@
         EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
-        ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+        ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
                 rdUnnestedMessage.getFields()[0]);
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 0, globalSort, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 1,
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 1,
                 terminateWriter, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 2,
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 2,
                 finalAggregator, 0);
 
         /**
          * connect the insert/delete operator
          */
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 3, insertOp,
-                0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 3, insertOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 4, deleteOp,
-                0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 4, deleteOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 0, globalGby, 0);
@@ -408,7 +406,7 @@
         EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
-        ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+        ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
                 rdUnnestedMessage.getFields()[0]);
 
@@ -417,16 +415,16 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, materializeRead, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materializeRead, 0, join, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 0, globalSort, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 1,
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 1,
                 terminateWriter, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 2,
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 2,
                 finalAggregator, 0);
         /**
          * connect the insert/delete operator
          */
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 3, insertOp, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 3, insertOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 4, deleteOp, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 4, deleteOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 0, globalGby, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
index e653196..0c3db38 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
@@ -195,7 +195,7 @@
         TerminationStateWriterOperatorDescriptor terminateWriter = new TerminationStateWriterOperatorDescriptor(spec,
                 configurationFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, terminateWriter, 1);
-        ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+        ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
 
         /**
          * final aggregate write operator
@@ -239,18 +239,16 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, localSort, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 1,
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 1,
                 terminateWriter, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 2,
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 2,
                 finalAggregator, 0);
         /**
          * connect the insert/delete operator
          */
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 3, insertOp,
-                0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 3, insertOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 4, deleteOp,
-                0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 4, deleteOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
@@ -441,7 +439,7 @@
         EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
-        ITuplePartitionComputerFactory hashPartitionComputerFactory = new MergePartitionComputerFactory();
+        ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
                 rdUnnestedMessage.getFields()[0]);
 
@@ -450,16 +448,16 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, materializeRead, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materializeRead, 0, join, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, localSort, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 1,
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 1,
                 terminateWriter, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 2,
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 2,
                 finalAggregator, 0);
         /**
          * connect the insert/delete operator
          */
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 3, insertOp, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 3, insertOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), join, 4, deleteOp, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 4, deleteOp, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
index 2a2e2bf..d099645 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
@@ -26,6 +26,7 @@
 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
 import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
+import edu.uci.ics.pregelix.runtime.bootstrap.NCApplicationEntryPoint;
 
 public class PregelixHyracksIntegrationUtil {
 
@@ -64,7 +65,9 @@
         ncConfig1.clusterNetIPAddress = "localhost";
         ncConfig1.ccPort = TEST_HYRACKS_CC_PORT;
         ncConfig1.dataIPAddress = "127.0.0.1";
+        ncConfig1.datasetIPAddress = "127.0.0.1";
         ncConfig1.nodeId = NC1_ID;
+        ncConfig1.appNCMainClass = NCApplicationEntryPoint.class.getName();
         nc1 = new NodeControllerService(ncConfig1);
         nc1.start();
 
@@ -73,7 +76,9 @@
         ncConfig2.clusterNetIPAddress = "localhost";
         ncConfig2.ccPort = TEST_HYRACKS_CC_PORT;
         ncConfig2.dataIPAddress = "127.0.0.1";
+        ncConfig2.datasetIPAddress = "127.0.0.1";
         ncConfig2.nodeId = NC2_ID;
+        ncConfig2.appNCMainClass = NCApplicationEntryPoint.class.getName();
         nc2 = new NodeControllerService(ncConfig2);
         nc2.start();
 
@@ -82,14 +87,6 @@
         ClusterConfig.loadClusterConfig(CC_HOST, TEST_HYRACKS_CC_CLIENT_PORT);
     }
 
-    public static void destroyApp(String hyracksAppName) throws Exception {
-        hcc.destroyApplication(hyracksAppName);
-    }
-
-    public static void createApp(String hyracksAppName) throws Exception {
-        hcc.createApplication(hyracksAppName, null);
-    }
-
     public static void deinit() throws Exception {
         nc2.stop();
         nc1.stop();
@@ -98,7 +95,7 @@
 
     public static void runJob(JobSpecification spec, String appName) throws Exception {
         spec.setFrameSize(FRAME_SIZE);
-        JobId jobId = hcc.startJob(appName, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+        JobId jobId = hcc.startJob(spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
         hcc.waitForCompletion(jobId);
     }
 
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/getip.sh b/pregelix/pregelix-core/src/main/resources/scripts/getip.sh
index e0cdf73..a691c0f 100755
--- a/pregelix/pregelix-core/src/main/resources/scripts/getip.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/getip.sh
@@ -6,6 +6,10 @@
 then
         #Get IP Address
         IPADDR=`/sbin/ifconfig eth0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+    	if [ "$IPADDR" = "" ]
+        then
+		IPADDR=`/sbin/ifconfig em1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+        fi 
 	if [ "$IPADDR" = "" ]
         then
 		IPADDR=`/sbin/ifconfig lo | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh b/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
index fe2551d..efb79ce 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
@@ -22,4 +22,4 @@
 
 #Launch hyracks cc script
 chmod -R 755 $HYRACKS_HOME
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 3 &> $CCLOGS_DIR/cc.log &
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh b/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh
index 6e0f90e..b059aad 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh
@@ -46,4 +46,4 @@
 cd $NCTMP_DIR
 
 #Launch hyracks nc
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -result-ip-address $IPADDR  -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh b/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh
index 03ce4e7..35c4794 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh
@@ -5,6 +5,10 @@
 PID=`ps -ef|grep ${USER}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
 
 if [ "$PID" == "" ]; then
+  PID=`ps -ef|grep ${USER}|grep java|grep 'hyracks'|awk '{print $2}'`
+fi
+
+if [ "$PID" == "" ]; then
   USERID=`id | sed 's/^uid=//;s/(.*$//'`
   PID=`ps -ef|grep ${USERID}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
 fi
diff --git a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
index a5cb6e2..3c00cad 100644
--- a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
+++ b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
@@ -103,7 +103,6 @@
         ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
         cleanupStores();
         PregelixHyracksIntegrationUtil.init();
-        PregelixHyracksIntegrationUtil.createApp(HYRACKS_APP_NAME);
 
         FileUtils.forceMkdir(new File(EXPECT_RESULT_DIR));
         FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
@@ -121,7 +120,6 @@
         runIndexRightOuterJoin();
         TestUtils.compareWithResult(new File(EXPECTED_RESULT_FILE), new File(ACTUAL_RESULT_FILE));
 
-        PregelixHyracksIntegrationUtil.destroyApp(HYRACKS_APP_NAME);
         PregelixHyracksIntegrationUtil.deinit();
     }
 
diff --git a/pregelix/pregelix-dataflow-std-base/pom.xml b/pregelix/pregelix-dataflow-std-base/pom.xml
index caa3222..4fda45f 100644
--- a/pregelix/pregelix-dataflow-std-base/pom.xml
+++ b/pregelix/pregelix-dataflow-std-base/pom.xml
@@ -42,8 +42,9 @@
 				</configuration>
 			</plugin>
 			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-clean-plugin</artifactId>
-				<version>2.5</version>
+                <version>2.4.1</version>
 				<configuration>
 					<filesets>
 						<fileset>
diff --git a/pregelix/pregelix-dataflow-std/pom.xml b/pregelix/pregelix-dataflow-std/pom.xml
index c5a0913..efe1607 100644
--- a/pregelix/pregelix-dataflow-std/pom.xml
+++ b/pregelix/pregelix-dataflow-std/pom.xml
@@ -43,8 +43,9 @@
 				</configuration>
 			</plugin>
 			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-clean-plugin</artifactId>
-				<version>2.5</version>
+				<version>2.4.1</version>
 				<configuration>
 					<filesets>
 						<fileset>
diff --git a/pregelix/pregelix-dataflow/pom.xml b/pregelix/pregelix-dataflow/pom.xml
index d3f396e..2d0859b 100644
--- a/pregelix/pregelix-dataflow/pom.xml
+++ b/pregelix/pregelix-dataflow/pom.xml
@@ -43,8 +43,9 @@
 				</configuration>
 			</plugin>
 			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-clean-plugin</artifactId>
-				<version>2.5</version>
+				<version>2.4.1</version>
 				<configuration>
 					<filesets>
 						<fileset>
diff --git a/pregelix/pregelix-example/pom.xml b/pregelix/pregelix-example/pom.xml
index e643331..84feb78 100644
--- a/pregelix/pregelix-example/pom.xml
+++ b/pregelix/pregelix-example/pom.xml
@@ -24,6 +24,7 @@
 			</plugin>
 			<plugin>
 				<artifactId>maven-assembly-plugin</artifactId>
+                <version>2.2-beta-5</version>
 				<configuration>
 					<descriptorRefs>
 						<descriptorRef>jar-with-dependencies</descriptorRef>
@@ -42,7 +43,7 @@
 			<plugin>
 				<groupId>org.codehaus.mojo</groupId>
 				<artifactId>appassembler-maven-plugin</artifactId>
-				<version>1.3</version>
+                <version>1.3</version>
 				<executions>
 					<execution>
 						<configuration>
@@ -79,7 +80,7 @@
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-clean-plugin</artifactId>
-				<version>2.5</version>
+				<version>2.4.1</version>
 				<configuration>
 					<filesets>
 						<fileset>
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
index 5773602..e54373f 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
@@ -47,7 +47,7 @@
             if (newVertex == null) {
                 newVertex = new GraphMutationVertex();
             }
-            if (getVertexId().get() % 2 == 0) {
+            if (getVertexId().get() % 2 == 0 || getVertexId().get() % 3 == 0) {
                 deleteVertex(getVertexId());
             } else {
                 vid.set(100 * getVertexId().get());
@@ -59,7 +59,7 @@
         } else {
             if (getVertexId().get() % 190 == 0) {
                 deleteVertex(getVertexId());
-            } 
+            }
             voteToHalt();
         }
     }
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java
index 37f03a5..da6d564 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java
@@ -77,7 +77,6 @@
         ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
         cleanupStores();
         PregelixHyracksIntegrationUtil.init();
-        PregelixHyracksIntegrationUtil.createApp(HYRACKS_APP_NAME);
         LOGGER.info("Hyracks mini-cluster started");
         startHDFS();
         FileUtils.forceMkdir(new File(EXPECT_RESULT_DIR));
@@ -112,7 +111,6 @@
     }
 
     public void tearDown() throws Exception {
-        PregelixHyracksIntegrationUtil.destroyApp(HYRACKS_APP_NAME);
         PregelixHyracksIntegrationUtil.deinit();
         LOGGER.info("Hyracks mini-cluster shut down");
         cleanupHDFS();
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestSuite.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestSuite.java
index 4b4cc2e..4bf83e6 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestSuite.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestSuite.java
@@ -74,7 +74,6 @@
         ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
         cleanupStores();
         PregelixHyracksIntegrationUtil.init();
-        PregelixHyracksIntegrationUtil.createApp(HYRACKS_APP_NAME);
         LOGGER.info("Hyracks mini-cluster started");
         FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
         FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
@@ -126,7 +125,6 @@
     }
 
     public void tearDown() throws Exception {
-        PregelixHyracksIntegrationUtil.destroyApp(HYRACKS_APP_NAME);
         PregelixHyracksIntegrationUtil.deinit();
         LOGGER.info("Hyracks mini-cluster shut down");
         cleanupHDFS();
@@ -169,7 +167,7 @@
     public void run(TestResult result) {
         try {
             int testCount = countTestCases();
-            for (int i = 2; i == 2; i++) {
+            for (int i = 0; i < testCount; i++) {
                 // cleanupStores();
                 Test each = this.testAt(i);
                 if (result.shouldStop())
diff --git a/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result
index 3f2fd60..a30166c 100644
--- a/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result
+++ b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result
@@ -1,19 +1,13 @@
 1 0.0
-3 0.0
 5 0.0
 7 0.0
-9 0.0
 11 0.0
 13 0.0
-15 0.0
 17 0.0
 19 0.0
 100 0.0
-300 0.0
 500 0.0
 700 0.0
-900 0.0
 1100 0.0
 1300 0.0
-1500 0.0
 1700 0.0
diff --git a/pregelix/pregelix-runtime/pom.xml b/pregelix/pregelix-runtime/pom.xml
index e75f98b..94bda18 100644
--- a/pregelix/pregelix-runtime/pom.xml
+++ b/pregelix/pregelix-runtime/pom.xml
@@ -42,8 +42,9 @@
 				</configuration>
 			</plugin>
 			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-clean-plugin</artifactId>
-				<version>2.5</version>
+				<version>2.4.1</version>
 				<configuration>
 					<filesets>
 						<fileset>
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCApplicationEntryPoint.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCApplicationEntryPoint.java
new file mode 100644
index 0000000..fbebc66
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCApplicationEntryPoint.java
@@ -0,0 +1,23 @@
+package edu.uci.ics.pregelix.runtime.bootstrap;
+
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.application.INCApplicationEntryPoint;
+import edu.uci.ics.pregelix.dataflow.context.RuntimeContext;
+
+public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
+    @Override
+    public void start(INCApplicationContext ncAppCtx, String[] args) throws Exception {
+        RuntimeContext rCtx = new RuntimeContext(ncAppCtx);
+        ncAppCtx.setApplicationObject(rCtx);
+    }
+
+    @Override
+    public void notifyStartupComplete() throws Exception {
+
+    }
+
+    @Override
+    public void stop() throws Exception {
+
+    }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCBootstrapImpl.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCBootstrapImpl.java
deleted file mode 100644
index 76c725e..0000000
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCBootstrapImpl.java
+++ /dev/null
@@ -1,46 +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.pregelix.runtime.bootstrap;
-
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.application.INCApplicationContext;
-import edu.uci.ics.hyracks.api.application.INCBootstrap;
-import edu.uci.ics.pregelix.dataflow.context.RuntimeContext;
-
-public class NCBootstrapImpl implements INCBootstrap {
-    private static final Logger LOGGER = Logger.getLogger(NCBootstrapImpl.class.getName());
-    private INCApplicationContext appCtx;
-
-    @Override
-    public void start() throws Exception {
-        LOGGER.info("Starting NC Bootstrap");
-        RuntimeContext rCtx = new RuntimeContext(appCtx);
-        appCtx.setApplicationObject(rCtx);
-        LOGGER.info("Initialized RuntimeContext: " + rCtx);
-    }
-
-    @Override
-    public void stop() throws Exception {
-        LOGGER.info("Stopping NC Bootstrap");
-        RuntimeContext rCtx = (RuntimeContext) appCtx.getApplicationObject();
-        rCtx.close();
-    }
-
-    @Override
-    public void setApplicationContext(INCApplicationContext appCtx) {
-        this.appCtx = appCtx;
-    }
-}
\ No newline at end of file