merge fullstack_dynamic_deployment
diff --git a/.gitignore b/.gitignore
new file mode 100644
index 0000000..db9db2a
--- /dev/null
+++ b/.gitignore
@@ -0,0 +1,318 @@
+.project
+.settings/
+algebricks/.project
+algebricks/.settings/
+algebricks/algebricks-common/.classpath
+algebricks/algebricks-common/.project
+algebricks/algebricks-common/.settings/
+algebricks/algebricks-common/target/
+algebricks/algebricks-compiler/.classpath
+algebricks/algebricks-compiler/.project
+algebricks/algebricks-compiler/.settings/
+algebricks/algebricks-compiler/target/
+algebricks/algebricks-core/.classpath
+algebricks/algebricks-core/.project
+algebricks/algebricks-core/.settings/
+algebricks/algebricks-core/target/
+algebricks/algebricks-data/.classpath
+algebricks/algebricks-data/.project
+algebricks/algebricks-data/.settings/
+algebricks/algebricks-data/target/
+algebricks/algebricks-examples/.project
+algebricks/algebricks-examples/.settings/
+algebricks/algebricks-examples/piglet-example/.classpath
+algebricks/algebricks-examples/piglet-example/.project
+algebricks/algebricks-examples/piglet-example/.settings/
+algebricks/algebricks-examples/piglet-example/target/
+algebricks/algebricks-rewriter/.classpath
+algebricks/algebricks-rewriter/.project
+algebricks/algebricks-rewriter/.settings/
+algebricks/algebricks-rewriter/target/
+algebricks/algebricks-runtime/.classpath
+algebricks/algebricks-runtime/.project
+algebricks/algebricks-runtime/.settings/
+algebricks/algebricks-runtime/target/
+algebricks/algebricks-tests/.classpath
+algebricks/algebricks-tests/.project
+algebricks/algebricks-tests/.settings/
+algebricks/algebricks-tests/ClusterControllerService/
+algebricks/algebricks-tests/target/
+hivesterix/.project
+hivesterix/.settings/
+hivesterix/hivesterix-common/.classpath
+hivesterix/hivesterix-common/.project
+hivesterix/hivesterix-common/.settings/
+hivesterix/hivesterix-common/target/
+hivesterix/hivesterix-dist/.classpath
+hivesterix/hivesterix-dist/.project
+hivesterix/hivesterix-dist/.settings/
+hivesterix/hivesterix-dist/ClusterControllerService/
+hivesterix/hivesterix-dist/build/
+hivesterix/hivesterix-dist/derby.log
+hivesterix/hivesterix-dist/hadoop-conf-tmp/
+hivesterix/hivesterix-dist/metastore_db/
+hivesterix/hivesterix-dist/target/
+hivesterix/hivesterix-optimizer/.classpath
+hivesterix/hivesterix-optimizer/.project
+hivesterix/hivesterix-optimizer/.settings/
+hivesterix/hivesterix-optimizer/target/
+hivesterix/hivesterix-runtime/.classpath
+hivesterix/hivesterix-runtime/.project
+hivesterix/hivesterix-runtime/.settings/
+hivesterix/hivesterix-runtime/target/
+hivesterix/hivesterix-serde/.classpath
+hivesterix/hivesterix-serde/.project
+hivesterix/hivesterix-serde/.settings/
+hivesterix/hivesterix-serde/target/
+hivesterix/hivesterix-translator/.classpath
+hivesterix/hivesterix-translator/.project
+hivesterix/hivesterix-translator/.settings/
+hivesterix/hivesterix-translator/target/
+hyracks/hyracks-api/.classpath
+hyracks/hyracks-api/.project
+hyracks/hyracks-api/.settings/
+hyracks/hyracks-api/target/
+hyracks/hyracks-client/.classpath
+hyracks/hyracks-client/.project
+hyracks/hyracks-client/.settings/
+hyracks/hyracks-client/target/
+hyracks/hyracks-comm/.classpath
+hyracks/hyracks-comm/.project
+hyracks/hyracks-comm/.settings/
+hyracks/hyracks-comm/target/
+hyracks/hyracks-control/.project
+hyracks/hyracks-control/.settings/
+hyracks/hyracks-control/hyracks-control-cc/.classpath
+hyracks/hyracks-control/hyracks-control-cc/.project
+hyracks/hyracks-control/hyracks-control-cc/.settings/
+hyracks/hyracks-control/hyracks-control-cc/target/
+hyracks/hyracks-control/hyracks-control-common/.classpath
+hyracks/hyracks-control/hyracks-control-common/.project
+hyracks/hyracks-control/hyracks-control-common/.settings/
+hyracks/hyracks-control/hyracks-control-common/target/
+hyracks/hyracks-control/hyracks-control-nc/.classpath
+hyracks/hyracks-control/hyracks-control-nc/.project
+hyracks/hyracks-control/hyracks-control-nc/.settings/
+hyracks/hyracks-control/hyracks-control-nc/target/
+hyracks/hyracks-data/.project
+hyracks/hyracks-data/.settings/
+hyracks/hyracks-data/hyracks-data-std/.classpath
+hyracks/hyracks-data/hyracks-data-std/.project
+hyracks/hyracks-data/hyracks-data-std/.settings/
+hyracks/hyracks-data/hyracks-data-std/target/
+hyracks/hyracks-dataflow-common/.classpath
+hyracks/hyracks-dataflow-common/.project
+hyracks/hyracks-dataflow-common/.settings/
+hyracks/hyracks-dataflow-common/target/
+hyracks/hyracks-dataflow-hadoop/.classpath
+hyracks/hyracks-dataflow-hadoop/.project
+hyracks/hyracks-dataflow-hadoop/.settings/
+hyracks/hyracks-dataflow-hadoop/target/
+hyracks/hyracks-dataflow-std/.classpath
+hyracks/hyracks-dataflow-std/.project
+hyracks/hyracks-dataflow-std/.settings/
+hyracks/hyracks-dataflow-std/target/
+hyracks/hyracks-dist/.classpath
+hyracks/hyracks-dist/.project
+hyracks/hyracks-dist/.settings/
+hyracks/hyracks-dist/target/
+hyracks/hyracks-documentation/.classpath
+hyracks/hyracks-documentation/.project
+hyracks/hyracks-documentation/.settings/
+hyracks/hyracks-documentation/target/
+hyracks/hyracks-examples/.project
+hyracks/hyracks-examples/.settings/
+hyracks/hyracks-examples/btree-example/.project
+hyracks/hyracks-examples/btree-example/.settings/
+hyracks/hyracks-examples/btree-example/btreeclient/.classpath
+hyracks/hyracks-examples/btree-example/btreeclient/.project
+hyracks/hyracks-examples/btree-example/btreeclient/.settings/
+hyracks/hyracks-examples/btree-example/btreeclient/target/
+hyracks/hyracks-examples/btree-example/btreehelper/.classpath
+hyracks/hyracks-examples/btree-example/btreehelper/.project
+hyracks/hyracks-examples/btree-example/btreehelper/.settings/
+hyracks/hyracks-examples/btree-example/btreehelper/target/
+hyracks/hyracks-examples/btree-example/btreeserver/.classpath
+hyracks/hyracks-examples/btree-example/btreeserver/.project
+hyracks/hyracks-examples/btree-example/btreeserver/.settings/
+hyracks/hyracks-examples/btree-example/btreeserver/target/
+hyracks/hyracks-examples/hadoop-compat-example/.project
+hyracks/hyracks-examples/hadoop-compat-example/.settings/
+hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/.classpath
+hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/.project
+hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/.settings/
+hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/target/
+hyracks/hyracks-examples/hadoop-compat-example/hadoopcompathelper/.classpath
+hyracks/hyracks-examples/hadoop-compat-example/hadoopcompathelper/.project
+hyracks/hyracks-examples/hadoop-compat-example/hadoopcompathelper/.settings/
+hyracks/hyracks-examples/hadoop-compat-example/hadoopcompathelper/target/
+hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/.classpath
+hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/.project
+hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/.settings/
+hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/target/
+hyracks/hyracks-examples/hyracks-integration-tests/.classpath
+hyracks/hyracks-examples/hyracks-integration-tests/.project
+hyracks/hyracks-examples/hyracks-integration-tests/.settings/
+hyracks/hyracks-examples/hyracks-integration-tests/target/
+hyracks/hyracks-examples/text-example/.project
+hyracks/hyracks-examples/text-example/.settings/
+hyracks/hyracks-examples/text-example/textclient/.classpath
+hyracks/hyracks-examples/text-example/textclient/.project
+hyracks/hyracks-examples/text-example/textclient/.settings/
+hyracks/hyracks-examples/text-example/textclient/target/
+hyracks/hyracks-examples/text-example/texthelper/.classpath
+hyracks/hyracks-examples/text-example/texthelper/.project
+hyracks/hyracks-examples/text-example/texthelper/.settings/
+hyracks/hyracks-examples/text-example/texthelper/target/
+hyracks/hyracks-examples/text-example/textserver/.classpath
+hyracks/hyracks-examples/text-example/textserver/.project
+hyracks/hyracks-examples/text-example/textserver/.settings/
+hyracks/hyracks-examples/text-example/textserver/target/
+hyracks/hyracks-examples/tpch-example/.project
+hyracks/hyracks-examples/tpch-example/.settings/
+hyracks/hyracks-examples/tpch-example/tpchclient/.classpath
+hyracks/hyracks-examples/tpch-example/tpchclient/.project
+hyracks/hyracks-examples/tpch-example/tpchclient/.settings/
+hyracks/hyracks-examples/tpch-example/tpchclient/target/
+hyracks/hyracks-examples/tpch-example/tpchserver/.classpath
+hyracks/hyracks-examples/tpch-example/tpchserver/.project
+hyracks/hyracks-examples/tpch-example/tpchserver/.settings/
+hyracks/hyracks-examples/tpch-example/tpchserver/target/
+hyracks/hyracks-hadoop-compat/.classpath
+hyracks/hyracks-hadoop-compat/.project
+hyracks/hyracks-hadoop-compat/.settings/
+hyracks/hyracks-hadoop-compat/target/
+hyracks/hyracks-hdfs/.project
+hyracks/hyracks-hdfs/.settings/
+hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/.classpath
+hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/.project
+hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/.settings/
+hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/target/
+hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/.classpath
+hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/.project
+hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/.settings/
+hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/target/
+hyracks/hyracks-hdfs/hyracks-hdfs-core/.classpath
+hyracks/hyracks-hdfs/hyracks-hdfs-core/.project
+hyracks/hyracks-hdfs/hyracks-hdfs-core/.settings/
+hyracks/hyracks-hdfs/hyracks-hdfs-core/ClusterControllerService/
+hyracks/hyracks-hdfs/hyracks-hdfs-core/actual/
+hyracks/hyracks-hdfs/hyracks-hdfs-core/build/
+hyracks/hyracks-hdfs/hyracks-hdfs-core/target/
+hyracks/hyracks-ipc/.classpath
+hyracks/hyracks-ipc/.project
+hyracks/hyracks-ipc/.settings/
+hyracks/hyracks-ipc/target/
+hyracks/hyracks-maven-plugins/.project
+hyracks/hyracks-maven-plugins/.settings/
+hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/.classpath
+hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/.project
+hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/.settings/
+hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/target/
+hyracks/hyracks-net/.classpath
+hyracks/hyracks-net/.project
+hyracks/hyracks-net/.settings/
+hyracks/hyracks-net/target/
+hyracks/hyracks-server/.classpath
+hyracks/hyracks-server/.project
+hyracks/hyracks-server/.settings/
+hyracks/hyracks-server/target/
+hyracks/hyracks-storage-am-bloomfilter/
+hyracks/hyracks-storage-am-btree/.classpath
+hyracks/hyracks-storage-am-btree/.project
+hyracks/hyracks-storage-am-btree/.settings/
+hyracks/hyracks-storage-am-btree/target/
+hyracks/hyracks-storage-am-common/.classpath
+hyracks/hyracks-storage-am-common/.project
+hyracks/hyracks-storage-am-common/.settings/
+hyracks/hyracks-storage-am-common/target/
+hyracks/hyracks-storage-am-invertedindex/.classpath
+hyracks/hyracks-storage-am-invertedindex/.project
+hyracks/hyracks-storage-am-invertedindex/.settings/
+hyracks/hyracks-storage-am-invertedindex/target/
+hyracks/hyracks-storage-am-lsm-btree/
+hyracks/hyracks-storage-am-lsm-common/
+hyracks/hyracks-storage-am-lsm-invertedindex/
+hyracks/hyracks-storage-am-lsm-rtree/
+hyracks/hyracks-storage-am-rtree/.classpath
+hyracks/hyracks-storage-am-rtree/.project
+hyracks/hyracks-storage-am-rtree/.settings/
+hyracks/hyracks-storage-am-rtree/target/
+hyracks/hyracks-storage-common/.classpath
+hyracks/hyracks-storage-common/.project
+hyracks/hyracks-storage-common/.settings/
+hyracks/hyracks-storage-common/target/
+hyracks/hyracks-test-support/.classpath
+hyracks/hyracks-test-support/.project
+hyracks/hyracks-test-support/.settings/
+hyracks/hyracks-test-support/target/
+hyracks/hyracks-tests/.project
+hyracks/hyracks-tests/.settings/
+hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/
+hyracks/hyracks-tests/hyracks-storage-am-btree-test/.classpath
+hyracks/hyracks-tests/hyracks-storage-am-btree-test/.project
+hyracks/hyracks-tests/hyracks-storage-am-btree-test/.settings/
+hyracks/hyracks-tests/hyracks-storage-am-btree-test/target/
+hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.classpath
+hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.project
+hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/.settings/
+hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/target/
+hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/
+hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/
+hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/
+hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/
+hyracks/hyracks-tests/hyracks-storage-am-rtree-test/.classpath
+hyracks/hyracks-tests/hyracks-storage-am-rtree-test/.project
+hyracks/hyracks-tests/hyracks-storage-am-rtree-test/.settings/
+hyracks/hyracks-tests/hyracks-storage-am-rtree-test/target/
+hyracks/hyracks-tests/hyracks-storage-common-test/.classpath
+hyracks/hyracks-tests/hyracks-storage-common-test/.project
+hyracks/hyracks-tests/hyracks-storage-common-test/.settings/
+hyracks/hyracks-tests/hyracks-storage-common-test/target/
+pregelix/.project
+pregelix/.settings/
+pregelix/pregelix-api/.classpath
+pregelix/pregelix-api/.project
+pregelix/pregelix-api/.settings/
+pregelix/pregelix-api/target/
+pregelix/pregelix-core/.classpath
+pregelix/pregelix-core/.project
+pregelix/pregelix-core/.settings/
+pregelix/pregelix-core/ClusterControllerService/
+pregelix/pregelix-core/actual/
+pregelix/pregelix-core/expected/
+pregelix/pregelix-core/target/
+pregelix/pregelix-core/teststore/
+pregelix/pregelix-dataflow-std-base/.classpath
+pregelix/pregelix-dataflow-std-base/.project
+pregelix/pregelix-dataflow-std-base/.settings/
+pregelix/pregelix-dataflow-std-base/target/
+pregelix/pregelix-dataflow-std/.classpath
+pregelix/pregelix-dataflow-std/.project
+pregelix/pregelix-dataflow-std/.settings/
+pregelix/pregelix-dataflow-std/target/
+pregelix/pregelix-dataflow/.classpath
+pregelix/pregelix-dataflow/.project
+pregelix/pregelix-dataflow/.settings/
+pregelix/pregelix-dataflow/target/
+pregelix/pregelix-dist/.classpath
+pregelix/pregelix-dist/.project
+pregelix/pregelix-dist/.settings/
+pregelix/pregelix-dist/target/
+pregelix/pregelix-example/.classpath
+pregelix/pregelix-example/.project
+pregelix/pregelix-example/.settings/
+pregelix/pregelix-example/ClusterControllerService/
+pregelix/pregelix-example/actual/
+pregelix/pregelix-example/build/
+pregelix/pregelix-example/edu.uci.ics.hyracks.control.nc.NodeControllerService/
+pregelix/pregelix-example/exception
+pregelix/pregelix-example/expected/
+pregelix/pregelix-example/target/
+pregelix/pregelix-example/teststore1/
+pregelix/pregelix-example/teststore2/
+pregelix/pregelix-runtime/.classpath
+pregelix/pregelix-runtime/.project
+pregelix/pregelix-runtime/.settings/
+pregelix/pregelix-runtime/target/
+
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-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
index 7e27b79..0d20f28 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
@@ -54,7 +54,8 @@
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-        if (op.getOperatorTag() != LogicalOperatorTag.WRITE && op.getOperatorTag() != LogicalOperatorTag.WRITE_RESULT) {
+        if (op.getOperatorTag() != LogicalOperatorTag.WRITE && op.getOperatorTag() != LogicalOperatorTag.WRITE_RESULT
+                && op.getOperatorTag() != LogicalOperatorTag.DISTRIBUTE_RESULT) {
             return false;
         }
         if (!roots.contains(op))
@@ -66,7 +67,8 @@
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-        if (op.getOperatorTag() != LogicalOperatorTag.WRITE && op.getOperatorTag() != LogicalOperatorTag.WRITE_RESULT) {
+        if (op.getOperatorTag() != LogicalOperatorTag.WRITE && op.getOperatorTag() != LogicalOperatorTag.WRITE_RESULT
+                && op.getOperatorTag() != LogicalOperatorTag.DISTRIBUTE_RESULT) {
             return false;
         }
         boolean rewritten = false;
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 dad7cd0..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
@@ -71,7 +71,6 @@
         nc2.start();
 
         hcc = new HyracksConnection(ccConfig.clientNetIpAddress, ccConfig.clientNetPort);
-        hcc.createApplication(AlgebricksConfig.HYRACKS_APP_NAME, null);
     }
 
     public static void deinit() throws Exception {
@@ -82,7 +81,7 @@
 
     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);
     }
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 379737f..e075f09 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
@@ -558,9 +558,8 @@
             }

         }

 

-        String applicationName = "hivesterix";

         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/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 c882742..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();

@@ -119,9 +116,6 @@
             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 09b632a..5e2e64e 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
@@ -27,8 +27,6 @@
 import org.apache.hadoop.mapred.JobConf;

 

 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;

@@ -116,7 +114,6 @@
         }

         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();

@@ -143,9 +140,6 @@
             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/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..6ee719b 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
@@ -16,25 +16,16 @@
 
 import java.io.Serializable;
 
+import edu.uci.ics.hyracks.api.job.IJobSerializerDeserializerContainer;
 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.
      * 
@@ -46,5 +37,6 @@
 
     public IMessageBroker getMessageBroker();
 
-    public String getApplicationName();
+    public IJobSerializerDeserializerContainer getJobSerializerDeserializerContainer();
+
 }
\ 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/client/HyracksClientInterfaceFunctions.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
index cd2b698..48dfb1c 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
@@ -15,10 +15,13 @@
 package edu.uci.ics.hyracks.api.client;
 
 import java.io.Serializable;
+import java.net.URL;
 import java.util.EnumSet;
+import java.util.List;
 
 import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
 import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 
@@ -26,9 +29,6 @@
     public enum FunctionId {
         GET_CLUSTER_CONTROLLER_INFO,
         GET_CLUSTER_TOPOLOGY,
-        CREATE_APPLICATION,
-        START_APPLICATION,
-        DESTROY_APPLICATION,
         CREATE_JOB,
         GET_JOB_STATUS,
         START_JOB,
@@ -37,7 +37,9 @@
         GET_DATASET_RECORD_DESCRIPTOR,
         GET_DATASET_RESULT_LOCATIONS,
         WAIT_FOR_COMPLETION,
-        GET_NODE_CONTROLLERS_INFO
+        GET_NODE_CONTROLLERS_INFO,
+        CLI_DEPLOY_BINARY,
+        CLI_UNDEPLOY_BINARY
     }
 
     public abstract static class Function implements Serializable {
@@ -55,63 +57,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;
 
@@ -134,14 +79,20 @@
     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;
+        private final DeploymentId deploymentId;
 
-        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;
+            this.deploymentId = null;
+        }
+
+        public StartJobFunction(DeploymentId deploymentId, byte[] acggfBytes, EnumSet<JobFlag> jobFlags) {
+            this.acggfBytes = acggfBytes;
+            this.jobFlags = jobFlags;
+            this.deploymentId = deploymentId;
         }
 
         @Override
@@ -149,10 +100,6 @@
             return FunctionId.START_JOB;
         }
 
-        public String getAppName() {
-            return appName;
-        }
-
         public byte[] getACGGFBytes() {
             return acggfBytes;
         }
@@ -160,6 +107,78 @@
         public EnumSet<JobFlag> getJobFlags() {
             return jobFlags;
         }
+
+        public DeploymentId getDeploymentId() {
+            return deploymentId;
+        }
+    }
+
+    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 GetDatasetDirectoryServiceInfoFunction extends Function {
@@ -266,4 +285,46 @@
             return FunctionId.GET_CLUSTER_TOPOLOGY;
         }
     }
+
+    public static class CliDeployBinaryFunction extends Function {
+        private static final long serialVersionUID = 1L;
+        private final List<URL> binaryURLs;
+        private final DeploymentId deploymentId;
+
+        public CliDeployBinaryFunction(List<URL> binaryURLs, DeploymentId deploymentId) {
+            this.binaryURLs = binaryURLs;
+            this.deploymentId = deploymentId;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.CLI_DEPLOY_BINARY;
+        }
+
+        public List<URL> getBinaryURLs() {
+            return binaryURLs;
+        }
+
+        public DeploymentId getDeploymentId() {
+            return deploymentId;
+        }
+    }
+
+    public static class CliUnDeployBinaryFunction extends Function {
+        private static final long serialVersionUID = 1L;
+        private final DeploymentId deploymentId;
+
+        public CliUnDeployBinaryFunction(DeploymentId deploymentId) {
+            this.deploymentId = deploymentId;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.CLI_UNDEPLOY_BINARY;
+        }
+
+        public DeploymentId getDeploymentId() {
+            return deploymentId;
+        }
+    }
 }
\ No newline at end of file
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 2ab42c0..36ceb18 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
@@ -14,10 +14,13 @@
  */
 package edu.uci.ics.hyracks.api.client;
 
+import java.net.URL;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Map;
 
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -42,27 +45,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);
@@ -70,9 +52,16 @@
     }
 
     @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 JobId startJob(DeploymentId deploymentId, byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception {
+        HyracksClientInterfaceFunctions.StartJobFunction sjf = new HyracksClientInterfaceFunctions.StartJobFunction(
+                deploymentId, acggfBytes, jobFlags);
         return (JobId) rpci.call(ipcHandle, sjf);
     }
 
@@ -100,4 +89,18 @@
         HyracksClientInterfaceFunctions.GetClusterTopologyFunction gctf = new HyracksClientInterfaceFunctions.GetClusterTopologyFunction();
         return (ClusterTopology) rpci.call(ipcHandle, gctf);
     }
+
+    @Override
+    public void deployBinary(List<URL> binaryURLs, DeploymentId deploymentId) throws Exception {
+        HyracksClientInterfaceFunctions.CliDeployBinaryFunction dbf = new HyracksClientInterfaceFunctions.CliDeployBinaryFunction(
+                binaryURLs, deploymentId);
+        rpci.call(ipcHandle, dbf);
+    }
+
+    @Override
+    public void unDeployBinary(DeploymentId deploymentId) throws Exception {
+        HyracksClientInterfaceFunctions.CliUnDeployBinaryFunction dbf = new HyracksClientInterfaceFunctions.CliUnDeployBinaryFunction(
+                deploymentId);
+        rpci.call(ipcHandle, dbf);
+    }
 }
\ No newline at end of file
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 e0fafb0..da14e9b 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
@@ -16,8 +16,12 @@
 
 import java.io.File;
 import java.net.InetSocketAddress;
+import java.net.URL;
+import java.util.ArrayList;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
@@ -27,6 +31,7 @@
 
 import edu.uci.ics.hyracks.api.client.impl.JobSpecificationActivityClusterGraphGeneratorFactory;
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
 import edu.uci.ics.hyracks.api.job.JobFlag;
@@ -77,46 +82,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();
     }
 
     public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception {
@@ -137,4 +124,55 @@
     public ClusterTopology getClusterTopology() throws Exception {
         return hci.getClusterTopology();
     }
+
+    @Override
+    public DeploymentId deployBinary(List<String> jars) throws Exception {
+        DeploymentId deploymentId = new DeploymentId(UUID.randomUUID().toString());
+        List<URL> binaryURLs = new ArrayList<URL>();
+        if (jars != null && jars.size() > 0) {
+            HttpClient hc = new DefaultHttpClient();
+            for (String jar : jars) {
+                int slashIndex = jar.lastIndexOf('/');
+                String fileName = jar.substring(slashIndex + 1);
+                String url = "http://" + ccHost + ":" + ccInfo.getWebPort() + "/applications/"
+                        + deploymentId.toString() + "&" + fileName;
+                HttpPut put = new HttpPut(url);
+                put.setEntity(new FileEntity(new File(jar), "application/octet-stream"));
+                HttpResponse response = hc.execute(put);
+                if (response != null) {
+                    response.getEntity().consumeContent();
+                }
+                if (response.getStatusLine().getStatusCode() != 200) {
+                    throw new HyracksException(response.getStatusLine().toString());
+                }
+                binaryURLs.add(new URL(url));
+            }
+        }
+        hci.deployBinary(binaryURLs, deploymentId);
+        return deploymentId;
+    }
+
+    @Override
+    public void unDeployBinary(DeploymentId deploymentId) throws Exception {
+        hci.unDeployBinary(deploymentId);
+    }
+
+    @Override
+    public JobId startJob(DeploymentId deploymentId, JobSpecification jobSpec) throws Exception {
+        return startJob(deploymentId, jobSpec, EnumSet.noneOf(JobFlag.class));
+    }
+
+    @Override
+    public JobId startJob(DeploymentId deploymentId, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags)
+            throws Exception {
+        JobSpecificationActivityClusterGraphGeneratorFactory jsacggf = new JobSpecificationActivityClusterGraphGeneratorFactory(
+                jobSpec);
+        return startJob(deploymentId, jsacggf, jobFlags);
+    }
+
+    @Override
+    public JobId startJob(DeploymentId deploymentId, IActivityClusterGraphGeneratorFactory acggf,
+            EnumSet<JobFlag> jobFlags) throws Exception {
+        return hci.startJob(deploymentId, JavaSerializationUtils.serialize(acggf), jobFlags);
+    }
 }
\ No newline at end of file
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 6333c22..4521149 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,11 +14,12 @@
  */
 package edu.uci.ics.hyracks.api.client;
 
-import java.io.File;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Map;
 
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
@@ -33,26 +34,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
@@ -71,7 +52,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.
@@ -84,7 +65,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.
@@ -97,8 +78,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;
 
     /**
      * Gets the IP Address and port for the DatasetDirectoryService wrapped in NetworkAddress
@@ -132,4 +120,60 @@
      * @throws Exception
      */
     public ClusterTopology getClusterTopology() throws Exception;
+
+    /**
+     * Deploy the user-defined jars to the cluster
+     * 
+     * @param jars
+     *            a list of user-defined jars
+     */
+    public DeploymentId deployBinary(List<String> jars) throws Exception;
+
+    /**
+     * undeploy a certain deployment
+     * 
+     * @param jars
+     *            a list of user-defined jars
+     */
+    public void unDeployBinary(DeploymentId deploymentId) throws Exception;
+
+    /**
+     * Start the specified Job.
+     * 
+     * @param deploymentId
+     *            the id of the specific deployment
+     * @param jobSpec
+     *            Job Specification
+     * @throws Exception
+     */
+    public JobId startJob(DeploymentId deploymentId, JobSpecification jobSpec) throws Exception;
+
+    /**
+     * Start the specified Job.
+     * 
+     * @param deploymentId
+     *            the id of the specific deployment
+     * @param jobSpec
+     *            Job Specification
+     * @param jobFlags
+     *            Flags
+     * @throws Exception
+     */
+    public JobId startJob(DeploymentId deploymentId, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags)
+            throws Exception;
+
+    /**
+     * Start the specified Job.
+     * 
+     * @param deploymentId
+     *            the id of the specific deployment
+     * @param acggf
+     *            Activity Cluster Graph Generator Factory
+     * @param jobFlags
+     *            Flags
+     * @throws Exception
+     */
+    public JobId startJob(DeploymentId deploymentId, IActivityClusterGraphGeneratorFactory acggf,
+            EnumSet<JobFlag> jobFlags) throws Exception;
+
 }
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 22b0a8f..e1a4185 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
@@ -14,10 +14,13 @@
  */
 package edu.uci.ics.hyracks.api.client;
 
+import java.net.URL;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Map;
 
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -26,15 +29,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 NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
 
@@ -43,4 +42,10 @@
     public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception;
 
     public ClusterTopology getClusterTopology() throws Exception;
+
+    public void deployBinary(List<URL> binaryURLs, DeploymentId deploymentId) throws Exception;
+
+    public void unDeployBinary(DeploymentId deploymentId) throws Exception;
+
+    public JobId startJob(DeploymentId deploymentId, byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception;
 }
\ 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/IHyracksJobletContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java
index fad4300..f8c5b6a 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java
@@ -28,4 +28,8 @@
     public ICounterContext getCounterContext();
 
     public Object getGlobalJobData();
+
+    public Class<?> loadClass(String className);
+
+    public ClassLoader getClassLoader();
 }
\ No newline at end of file
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 a2ee977..f0b47b1 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
@@ -16,6 +16,7 @@
 
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 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;
@@ -31,5 +32,5 @@
 
     public IDatasetPartitionManager getDatasetPartitionManager();
 
-    public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception;
+    public void sendApplicationMessageToCC(byte[] message, DeploymentId deploymendId, String nodeId) throws Exception;
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/DatasetJobRecord.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/DatasetJobRecord.java
new file mode 100644
index 0000000..dc99ef3
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/DatasetJobRecord.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import java.util.HashMap;
+
+public class DatasetJobRecord extends HashMap<ResultSetId, ResultSetMetaData> {
+    public enum Status {
+        RUNNING,
+        SUCCESS,
+        FAILED
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    private Status status;
+
+    public DatasetJobRecord() {
+        this.status = Status.RUNNING;
+    }
+
+    public void start() {
+        status = Status.RUNNING;
+    }
+
+    public void success() {
+        status = Status.SUCCESS;
+    }
+
+    public void fail() {
+        status = Status.FAILED;
+    }
+
+    public Status getStatus() {
+        return status;
+    }
+}
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
index 5266333..52e6005 100644
--- 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
@@ -15,11 +15,12 @@
 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.dataset.DatasetJobRecord.Status;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IJobLifecycleListener;
 import edu.uci.ics.hyracks.api.job.JobId;
 
-public interface IDatasetDirectoryService {
+public interface IDatasetDirectoryService extends IJobLifecycleListener {
     public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
             int nPartitions, NetworkAddress networkAddress);
 
@@ -27,6 +28,8 @@
 
     public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition);
 
+    public void reportJobFailure(JobId jobId);
+
     public Status getResultStatus(JobId jobId, ResultSetId rsId) throws HyracksDataException;
 
     public DatasetDirectoryRecord[] getResultPartitionLocations(JobId jobId, ResultSetId rsId,
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/ResultSetMetaData.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/ResultSetMetaData.java
new file mode 100644
index 0000000..e3ad69a
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/ResultSetMetaData.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;
+
+public 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-api/src/main/java/edu/uci/ics/hyracks/api/deployment/DeploymentId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/deployment/DeploymentId.java
new file mode 100644
index 0000000..b642bc7
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/deployment/DeploymentId.java
@@ -0,0 +1,33 @@
+package edu.uci.ics.hyracks.api.deployment;
+
+import java.io.Serializable;
+
+public class DeploymentId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final String deploymentKey;
+
+    public DeploymentId(String deploymentKey) {
+        this.deploymentKey = deploymentKey;
+    }
+
+    public int hashCode() {
+        return deploymentKey.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof DeploymentId)) {
+            return false;
+        }
+        return ((DeploymentId) o).deploymentKey.equals(deploymentKey);
+    }
+
+    @Override
+    public String toString() {
+        return deploymentKey;
+    }
+}
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/IJobSerializerDeserializer.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializer.java
new file mode 100644
index 0000000..d402cd3
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializer.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.job;
+
+import java.io.Serializable;
+import java.net.URL;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+
+public interface IJobSerializerDeserializer {
+
+    public Object deserialize(byte[] bytes) throws HyracksException;
+
+    public byte[] serialize(Serializable job) throws HyracksException;
+
+    public Class<?> loadClass(String className) throws HyracksException;
+
+    public void addClassPathURLs(List<URL> binaryURLs) throws HyracksException;
+
+    public ClassLoader getClassLoader() throws HyracksException;
+
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializerContainer.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializerContainer.java
new file mode 100644
index 0000000..0ce2346
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializerContainer.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.job;
+
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
+
+public interface IJobSerializerDeserializerContainer {
+
+    /**
+     * Get the IJobSerializerDeserializer implementation instance for a specific deployment id
+     * 
+     * @param deploymentId
+     * @return
+     */
+    public IJobSerializerDeserializer getJobSerializerDeerializer(DeploymentId deploymentId);
+
+    /**
+     * Add a deployment with the job serializer deserializer
+     * 
+     * @param deploymentId
+     * @param jobSerDe
+     */
+    public void addJobSerializerDeserializer(DeploymentId deploymentId, IJobSerializerDeserializer jobSerDe);
+
+    /**
+     * Remove a deployment
+     * 
+     * @param deploymentId
+     */
+    public void removeJobSerializerDeserializer(DeploymentId deploymentId);
+
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializer.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializer.java
new file mode 100644
index 0000000..0ea4309
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializer.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.job;
+
+import java.io.Serializable;
+import java.net.URL;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
+
+public class JobSerializerDeserializer implements IJobSerializerDeserializer {
+
+    @Override
+    public Object deserialize(byte[] jsBytes) throws HyracksException {
+        try {
+            return JavaSerializationUtils.deserialize(jsBytes);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    @Override
+    public byte[] serialize(Serializable obj) throws HyracksException {
+        try {
+            return JavaSerializationUtils.serialize(obj);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    @Override
+    public void addClassPathURLs(List<URL> binaryURLs) {
+        throw new UnsupportedOperationException("Not supported by " + this.getClass().getName());
+    }
+
+    @Override
+    public Class<?> loadClass(String className) throws HyracksException {
+        try {
+            return this.getClass().getClassLoader().loadClass(className);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    @Override
+    public ClassLoader getClassLoader() throws HyracksException {
+        return this.getClass().getClassLoader();
+    }
+
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializerContainer.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializerContainer.java
new file mode 100644
index 0000000..35a1e8b
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializerContainer.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.job;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
+
+public class JobSerializerDeserializerContainer implements IJobSerializerDeserializerContainer {
+
+    private IJobSerializerDeserializer defaultJobSerDe = new JobSerializerDeserializer();
+    private Map<DeploymentId, IJobSerializerDeserializer> jobSerializerDeserializerMap = new HashMap<DeploymentId, IJobSerializerDeserializer>();
+
+    @Override
+    public synchronized IJobSerializerDeserializer getJobSerializerDeerializer(DeploymentId deploymentId) {
+        if (deploymentId == null) {
+            return defaultJobSerDe;
+        }
+        IJobSerializerDeserializer jobSerDe = jobSerializerDeserializerMap.get(deploymentId);
+        return jobSerDe;
+    }
+
+    @Override
+    public synchronized void addJobSerializerDeserializer(DeploymentId deploymentId, IJobSerializerDeserializer jobSerDe) {
+        jobSerializerDeserializerMap.put(deploymentId, jobSerDe);
+    }
+
+    @Override
+    public synchronized void removeJobSerializerDeserializer(DeploymentId deploymentId) {
+        jobSerializerDeserializerMap.remove(deploymentId);
+    }
+
+}
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..a3c8aed 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,27 @@
         return baos.toByteArray();
     }
 
+    public static byte[] serialize(Serializable jobSpec, ClassLoader classLoader) throws IOException {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        ObjectOutputStream oos = new ObjectOutputStream(baos);
+        ClassLoader ctxCL = Thread.currentThread().getContextClassLoader();
+        try {
+            Thread.currentThread().setContextClassLoader(classLoader);
+            oos.writeObject(jobSpec);
+            return baos.toByteArray();
+        } finally {
+            Thread.currentThread().setContextClassLoader(ctxCL);
+        }
+    }
+
+    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;
@@ -47,6 +68,10 @@
         }
     }
 
+    public static Class<?> loadClass(String className) throws IOException, ClassNotFoundException {
+        return Class.forName(className);
+    }
+
     private static class ClassLoaderObjectInputStream extends ObjectInputStream {
         private ClassLoader classLoader;
 
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-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 82457fe..623c54f 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,14 +31,16 @@
 
 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.DatasetJobRecord.Status;
 import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.api.topology.ClusterTopology;
@@ -48,11 +49,9 @@
 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.CliDeployBinaryWork;
+import edu.uci.ics.hyracks.control.cc.work.CliUnDeployBinaryWork;
 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;
@@ -62,6 +61,7 @@
 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.NotifyDeployBinaryWork;
 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;
@@ -77,6 +77,7 @@
 import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
 import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentRun;
 import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions;
 import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions.Function;
 import edu.uci.ics.hyracks.control.common.logs.LogFile;
@@ -104,14 +105,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;
@@ -130,13 +131,14 @@
 
     private long jobCounter;
 
+    private final Map<DeploymentId, DeploymentRun> deploymentRunMap;
+
     public ClusterControllerService(final CCConfig ccConfig) throws Exception {
         this.ccConfig = ccConfig;
         File jobLogFolder = new File(ccConfig.ccRoot, "logs/jobs");
         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();
@@ -175,8 +177,10 @@
             }
         };
         sweeper = new DeadNodeSweeper();
-        datasetDirectoryService = new DatasetDirectoryService();
+        datasetDirectoryService = new DatasetDirectoryService(ccConfig.jobHistorySize);
         jobCounter = 0;
+
+        deploymentRunMap = new HashMap<DeploymentId, DeploymentRun>();
     }
 
     private static ClusterTopology computeClusterTopology(CCConfig ccConfig) throws Exception {
@@ -204,9 +208,23 @@
                 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);
+        appCtx.addJobLifecycleListener(datasetDirectoryService);
+        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");
@@ -226,10 +244,6 @@
         return ccContext;
     }
 
-    public Map<String, CCApplicationContext> getApplicationMap() {
-        return applications;
-    }
-
     public Map<JobId, JobRun> getActiveRunMap() {
         return activeRunMap;
     }
@@ -262,6 +276,10 @@
         return ccConfig;
     }
 
+    public CCApplicationContext getApplicationContext() {
+        return appCtx;
+    }
+
     private JobId createJobId() {
         return new JobId(jobCounter++);
     }
@@ -307,27 +325,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(),
@@ -338,7 +335,7 @@
                 case START_JOB: {
                     HyracksClientInterfaceFunctions.StartJobFunction sjf = (HyracksClientInterfaceFunctions.StartJobFunction) fn;
                     JobId jobId = createJobId();
-                    workQueue.schedule(new JobStartWork(ClusterControllerService.this, sjf.getAppName(), sjf
+                    workQueue.schedule(new JobStartWork(ClusterControllerService.this, sjf.getDeploymentId(), sjf
                             .getACGGFBytes(), sjf.getJobFlags(), jobId, new IPCResponder<JobId>(handle, mid)));
                     return;
                 }
@@ -364,6 +361,27 @@
                     return;
                 }
 
+                case GET_DATASET_DIRECTORY_SERIVICE_INFO: {
+                    workQueue.schedule(new GetDatasetDirectoryServiceInfoWork(ClusterControllerService.this,
+                            new IPCResponder<NetworkAddress>(handle, mid)));
+                    return;
+                }
+
+                case GET_DATASET_RESULT_STATUS: {
+                    HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction gdrlf = (HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction) fn;
+                    workQueue.schedule(new GetResultStatusWork(ClusterControllerService.this, gdrlf.getJobId(), gdrlf
+                            .getResultSetId(), new IPCResponder<Status>(handle, mid)));
+                    return;
+                }
+
+                case GET_DATASET_RESULT_LOCATIONS: {
+                    HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction gdrlf = (HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction) fn;
+                    workQueue.schedule(new GetResultPartitionLocationsWork(ClusterControllerService.this, gdrlf
+                            .getJobId(), gdrlf.getResultSetId(), gdrlf.getKnownRecords(),
+                            new IPCResponder<DatasetDirectoryRecord[]>(handle, mid)));
+                    return;
+                }
+
                 case WAIT_FOR_COMPLETION: {
                     HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = (HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
                     workQueue.schedule(new WaitForJobCompletionWork(ClusterControllerService.this, wfcf.getJobId(),
@@ -385,6 +403,20 @@
                     }
                     return;
                 }
+
+                case CLI_DEPLOY_BINARY: {
+                    HyracksClientInterfaceFunctions.CliDeployBinaryFunction dbf = (HyracksClientInterfaceFunctions.CliDeployBinaryFunction) fn;
+                    workQueue.schedule(new CliDeployBinaryWork(ClusterControllerService.this, dbf.getBinaryURLs(), dbf
+                            .getDeploymentId(), new IPCResponder<DeploymentId>(handle, mid)));
+                    return;
+                }
+
+                case CLI_UNDEPLOY_BINARY: {
+                    HyracksClientInterfaceFunctions.CliUnDeployBinaryFunction udbf = (HyracksClientInterfaceFunctions.CliUnDeployBinaryFunction) fn;
+                    workQueue.schedule(new CliUnDeployBinaryWork(ClusterControllerService.this, udbf.getDeploymentId(),
+                            new IPCResponder<DeploymentId>(handle, mid)));
+                    return;
+                }
             }
             try {
                 handle.send(mid, null, new IllegalArgumentException("Unknown function " + fn.getFunctionId()));
@@ -426,6 +458,13 @@
                     return;
                 }
 
+                case NOTIFY_DEPLOY_BINARY: {
+                    CCNCFunctions.NotifyDeployBinaryFunction ndbf = (CCNCFunctions.NotifyDeployBinaryFunction) fn;
+                    workQueue.schedule(new NotifyDeployBinaryWork(ClusterControllerService.this,
+                            ndbf.getDeploymentId(), ndbf.getNodeId(), ndbf.getDeploymentStatus()));
+                    return;
+                }
+
                 case REPORT_PROFILE: {
                     CCNCFunctions.ReportProfileFunction rpf = (CCNCFunctions.ReportProfileFunction) fn;
                     workQueue.schedule(new ReportProfilesWork(ClusterControllerService.this, rpf.getProfiles()));
@@ -481,15 +520,10 @@
                     return;
                 }
 
-                case APPLICATION_STATE_CHANGE_RESPONSE: {
-                    CCNCFunctions.ApplicationStateChangeResponseFunction astrf = (CCNCFunctions.ApplicationStateChangeResponseFunction) fn;
-                    workQueue.schedule(new ApplicationStateChangeWork(ClusterControllerService.this, astrf));
-                    return;
-                }
                 case SEND_APPLICATION_MESSAGE: {
                     CCNCFunctions.SendApplicationMessageFunction rsf = (CCNCFunctions.SendApplicationMessageFunction) fn;
                     workQueue.schedule(new ApplicationMessageWork(ClusterControllerService.this, rsf.getMessage(), rsf
-                            .getAppName(), rsf.getNodeId()));
+                            .getDeploymentId(), rsf.getNodeId()));
                     return;
                 }
 
@@ -513,4 +547,32 @@
             LOGGER.warning("Unknown function: " + fn.getFunctionId());
         }
     }
+
+    /**
+     * Add a deployment run
+     * 
+     * @param deploymentKey
+     * @param nodeControllerIds
+     */
+    public synchronized void addDeploymentRun(DeploymentId deploymentKey, DeploymentRun dRun) {
+        deploymentRunMap.put(deploymentKey, dRun);
+    }
+
+    /**
+     * Get a deployment run
+     * 
+     * @param deploymentKey
+     */
+    public synchronized DeploymentRun getDeploymentRun(DeploymentId deploymentKey) {
+        return deploymentRunMap.get(deploymentKey);
+    }
+
+    /**
+     * Remove a deployment run
+     * 
+     * @param deploymentKey
+     */
+    public synchronized void removeDeploymentRun(DeploymentId deploymentKey) {
+        deploymentRunMap.remove(deploymentKey);
+    }
 }
\ No newline at end of file
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
index 13d0c30..cdcdf4c 100644
--- 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
@@ -14,17 +14,20 @@
  */
 package edu.uci.ics.hyracks.control.cc.dataset;
 
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
+import java.util.LinkedHashMap;
 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.DatasetJobRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.Status;
 import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
 import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.dataset.ResultSetMetaData;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
 import edu.uci.ics.hyracks.api.job.JobId;
 
 /**
@@ -35,25 +38,46 @@
  * job.
  */
 public class DatasetDirectoryService implements IDatasetDirectoryService {
-    private final Map<JobId, Map<ResultSetId, ResultSetMetaData>> jobResultLocationsMap;
+    private final Map<JobId, DatasetJobRecord> jobResultLocations;
 
-    public DatasetDirectoryService() {
-        jobResultLocationsMap = new HashMap<JobId, Map<ResultSetId, ResultSetMetaData>>();
+    public DatasetDirectoryService(final int jobHistorySize) {
+        jobResultLocations = new LinkedHashMap<JobId, DatasetJobRecord>() {
+            private static final long serialVersionUID = 1L;
+
+            protected boolean removeEldestEntry(Map.Entry<JobId, DatasetJobRecord> eldest) {
+                return size() > jobHistorySize;
+            }
+        };
+    }
+
+    @Override
+    public void notifyJobCreation(JobId jobId, IActivityClusterGraphGeneratorFactory acggf) throws HyracksException {
+        DatasetJobRecord djr = jobResultLocations.get(jobId);
+        if (djr == null) {
+            djr = new DatasetJobRecord();
+            jobResultLocations.put(jobId, djr);
+        }
+    }
+
+    @Override
+    public void notifyJobStart(JobId jobId) throws HyracksException {
+        // Auto-generated method stub
+    }
+
+    @Override
+    public void notifyJobFinish(JobId jobId) throws HyracksException {
+        // Auto-generated method stub
     }
 
     @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);
-        }
+        DatasetJobRecord djr = jobResultLocations.get(jobId);
 
-        ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+        ResultSetMetaData resultSetMetaData = djr.get(rsId);
         if (resultSetMetaData == null) {
             resultSetMetaData = new ResultSetMetaData(orderedResult, new DatasetDirectoryRecord[nPartitions]);
-            rsMap.put(rsId, resultSetMetaData);
+            djr.put(rsId, resultSetMetaData);
         }
 
         DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
@@ -67,20 +91,42 @@
 
     @Override
     public synchronized void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) {
-        DatasetDirectoryRecord ddr = getDatasetDirectoryRecord(jobId, rsId, partition);
-        ddr.writeEOS();
+        int successCount = 0;
+
+        DatasetJobRecord djr = jobResultLocations.get(jobId);
+        ResultSetMetaData resultSetMetaData = djr.get(rsId);
+        DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+        records[partition].writeEOS();
+
+        for (DatasetDirectoryRecord record : records) {
+            if (record.getStatus() == DatasetDirectoryRecord.Status.SUCCESS) {
+                successCount++;
+            }
+        }
+        if (successCount == records.length) {
+            djr.success();
+        }
+        notifyAll();
     }
 
     @Override
     public synchronized void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) {
-        DatasetDirectoryRecord ddr = getDatasetDirectoryRecord(jobId, rsId, partition);
-        ddr.fail();
+        DatasetJobRecord djr = jobResultLocations.get(jobId);
+        djr.fail();
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void reportJobFailure(JobId jobId) {
+        DatasetJobRecord djr = jobResultLocations.get(jobId);
+        djr.fail();
+        notifyAll();
     }
 
     @Override
     public synchronized Status getResultStatus(JobId jobId, ResultSetId rsId) throws HyracksDataException {
-        Map<ResultSetId, ResultSetMetaData> rsMap;
-        while ((rsMap = jobResultLocationsMap.get(jobId)) == null) {
+        DatasetJobRecord djr;
+        while ((djr = jobResultLocations.get(jobId)) == null) {
             try {
                 wait();
             } catch (InterruptedException e) {
@@ -88,38 +134,7 @@
             }
         }
 
-        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;
+        return djr.getStatus();
     }
 
     @Override
@@ -136,13 +151,6 @@
         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
@@ -177,14 +185,19 @@
      */
     private DatasetDirectoryRecord[] updatedRecords(JobId jobId, ResultSetId rsId, DatasetDirectoryRecord[] knownRecords)
             throws HyracksDataException {
-        Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
-        if (rsMap == null) {
-            return null;
+        DatasetJobRecord djr = jobResultLocations.get(jobId);
+
+        if (djr == null) {
+            throw new HyracksDataException("Requested JobId " + jobId + "doesn't exist");
         }
 
-        ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+        if (djr.getStatus() == Status.FAILED) {
+            throw new HyracksDataException("Job failed.");
+        }
+
+        ResultSetMetaData resultSetMetaData = djr.get(rsId);
         if (resultSetMetaData == null || resultSetMetaData.getRecords() == null) {
-            throw new HyracksDataException("ResultSet locations uninitialized when it is expected to be initialized.");
+            return null;
         }
 
         boolean ordered = resultSetMetaData.getOrderedResult();
@@ -219,23 +232,4 @@
         }
         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..827a71e 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
@@ -29,6 +29,7 @@
 import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.ActivityCluster;
 import edu.uci.ics.hyracks.api.job.ActivityClusterGraph;
@@ -45,9 +46,9 @@
 import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
 
 public class JobRun implements IJobStatusConditionVariable {
-    private final JobId jobId;
+    private final DeploymentId deploymentId;
 
-    private final String applicationName;
+    private final JobId jobId;
 
     private final IActivityClusterGraphGenerator acgg;
 
@@ -83,10 +84,10 @@
 
     private Exception pendingException;
 
-    public JobRun(ClusterControllerService ccs, JobId jobId, String applicationName,
+    public JobRun(ClusterControllerService ccs, DeploymentId deploymentId, JobId jobId,
             IActivityClusterGraphGenerator acgg, EnumSet<JobFlag> jobFlags) {
+        this.deploymentId = deploymentId;
         this.jobId = jobId;
-        this.applicationName = applicationName;
         this.acgg = acgg;
         this.acg = acgg.initialize();
         this.scheduler = new JobScheduler(ccs, this, acgg.getConstraints());
@@ -99,12 +100,12 @@
         connectorPolicyMap = new HashMap<ConnectorDescriptorId, IConnectorPolicy>();
     }
 
-    public JobId getJobId() {
-        return jobId;
+    public DeploymentId getDeploymentId() {
+        return deploymentId;
     }
 
-    public String getApplicationName() {
-        return applicationName;
+    public JobId getJobId() {
+        return jobId;
     }
 
     public ActivityClusterGraph getActivityClusterGraph() {
@@ -208,7 +209,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..f3d7d34 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
@@ -34,6 +34,7 @@
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.ActivityCluster;
 import edu.uci.ics.hyracks.api.job.ActivityClusterGraph;
@@ -427,9 +428,9 @@
     }
 
     private void startTasks(Map<String, List<TaskAttemptDescriptor>> taskAttemptMap) throws HyracksException {
+        final DeploymentId deploymentId = jobRun.getDeploymentId();
         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,8 +445,8 @@
                 }
                 try {
                     byte[] jagBytes = changed ? JavaSerializationUtils.serialize(acg) : null;
-                    node.getNodeController().startTasks(appName, jobId, jagBytes, taskDescriptors, connectorPolicies,
-                            jobRun.getFlags());
+                    node.getNodeController().startTasks(deploymentId, 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
old mode 100644
new mode 100755
index 73f3a1a..0236c4b
--- 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
@@ -14,6 +14,9 @@
  */
 package edu.uci.ics.hyracks.control.cc.web;
 
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -22,13 +25,13 @@
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.commons.io.FileUtils;
 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 {
@@ -52,18 +55,21 @@
             if (parts.length != 1) {
                 return;
             }
-            final String appName = parts[0];
+            final String[] params = parts[0].split("&");
+            String deployIdString = params[0];
+            String rootDir = ccs.getServerContext().getBaseDir().toString();
+            final String deploymentDir = rootDir.endsWith(File.separator) ? rootDir + "applications/" + deployIdString
+                    : rootDir + File.separator + "/applications/" + File.separator + deployIdString;
             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();
-                        }
+                        FileUtils.forceMkdir(new File(deploymentDir));
+                        String fileName = params[1];
+                        File jarFile = new File(deploymentDir, fileName);
+                        os = new FileOutputStream(jarFile);
                     }
                 }
                 OutputStreamGetter r = new OutputStreamGetter();
@@ -83,11 +89,9 @@
 
                     @Override
                     protected void doRun() throws Exception {
-                        ApplicationContext appCtx;
-                        appCtx = ccs.getApplicationMap().get(appName);
-                        if (appCtx != null && appCtx.containsHar()) {
-                            is = appCtx.getHarInputStream();
-                        }
+                        String fileName = params[1];
+                        File jarFile = new File(deploymentDir, fileName);
+                        is = new FileInputStream(jarFile);
                     }
                 }
                 InputStreamGetter r = new InputStreamGetter();
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..50b3e30 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
@@ -14,49 +14,48 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
-import java.io.IOException;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.messages.IMessage;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentUtils;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
 /**
  * @author rico
- * 
  */
 public class ApplicationMessageWork extends AbstractWork {
 
     private static final Logger LOGGER = Logger.getLogger(ApplicationMessageWork.class.getName());
     private byte[] message;
+    private DeploymentId deploymentId;
     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, DeploymentId deploymentId, String nodeId) {
         this.ccs = ccs;
+        this.deploymentId = deploymentId;
         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) DeploymentUtils.deserialize(message, deploymentId, ctx);
+            ccs.getExecutor().execute(new Runnable() {
+                @Override
                 public void run() {
                     ctx.getMessageBroker().receivedMessage(data, nodeId);
                 }
-            }).start();
-        } catch (IOException e) {
+            });
+        } catch (Exception e) {
             LOGGER.log(Level.WARNING, "Error in stats reporting", e);
-        } catch (ClassNotFoundException e) {
-            Logger.getLogger(this.getClass().getName()).log(Level.WARNING, "Error in stats reporting", e);
+            throw new RuntimeException(e);
         }
     }
 
@@ -64,5 +63,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/CliDeployBinaryWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/CliDeployBinaryWork.java
new file mode 100644
index 0000000..fa44633
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/CliDeployBinaryWork.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.control.cc.work;
+
+import java.net.URL;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentRun;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentUtils;
+import edu.uci.ics.hyracks.control.common.work.IPCResponder;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class CliDeployBinaryWork extends SynchronizableWork {
+
+    private ClusterControllerService ccs;
+    private List<URL> binaryURLs;
+    private DeploymentId deploymentId;
+    private IPCResponder<DeploymentId> callback;
+
+    public CliDeployBinaryWork(ClusterControllerService ncs, List<URL> binaryURLs, DeploymentId deploymentId,
+            IPCResponder<DeploymentId> callback) {
+        this.ccs = ncs;
+        this.binaryURLs = binaryURLs;
+        this.deploymentId = deploymentId;
+        this.callback = callback;
+    }
+
+    @Override
+    public void doRun() {
+        try {
+            if (deploymentId == null) {
+                deploymentId = new DeploymentId(UUID.randomUUID().toString());
+            }
+            /**
+             * Deploy for the cluster controller
+             */
+            DeploymentUtils.deploy(deploymentId, binaryURLs, ccs.getApplicationContext()
+                    .getJobSerializerDeserializerContainer(), ccs.getServerContext(), false);
+
+            /**
+             * Deploy for the node controllers
+             */
+            Map<String, NodeControllerState> nodeControllerStateMap = ccs.getNodeMap();
+
+            Set<String> nodeIds = new TreeSet<String>();
+            for (String nc : nodeControllerStateMap.keySet()) {
+                nodeIds.add(nc);
+            }
+            final DeploymentRun dRun = new DeploymentRun(nodeIds);
+            ccs.addDeploymentRun(deploymentId, dRun);
+
+            /***
+             * deploy binaries to each node controller
+             */
+            for (NodeControllerState ncs : nodeControllerStateMap.values()) {
+                ncs.getNodeController().deployBinary(deploymentId, binaryURLs);
+            }
+
+            ccs.getExecutor().execute(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        /**
+                         * wait for completion
+                         */
+                        dRun.waitForCompletion();
+                        ccs.removeDeploymentRun(deploymentId);
+                        callback.setValue(deploymentId);
+                    } catch (Exception e) {
+                        callback.setException(e);
+                    }
+                }
+            });
+        } catch (Exception e) {
+            callback.setException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/CliUnDeployBinaryWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/CliUnDeployBinaryWork.java
new file mode 100644
index 0000000..8e429c9
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/CliUnDeployBinaryWork.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.control.cc.work;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentRun;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentUtils;
+import edu.uci.ics.hyracks.control.common.work.IPCResponder;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class CliUnDeployBinaryWork extends SynchronizableWork {
+
+    private ClusterControllerService ccs;
+    private DeploymentId deploymentId;
+    private IPCResponder<DeploymentId> callback;
+
+    public CliUnDeployBinaryWork(ClusterControllerService ncs, DeploymentId deploymentId,
+            IPCResponder<DeploymentId> callback) {
+        this.ccs = ncs;
+        this.deploymentId = deploymentId;
+        this.callback = callback;
+    }
+
+    @Override
+    public void doRun() {
+        try {
+            if (deploymentId == null) {
+                deploymentId = new DeploymentId(UUID.randomUUID().toString());
+            }
+            /**
+             * Deploy for the cluster controller
+             */
+            DeploymentUtils.undeploy(deploymentId, ccs.getApplicationContext().getJobSerializerDeserializerContainer(),
+                    ccs.getServerContext());
+
+            /**
+             * Deploy for the node controllers
+             */
+            Map<String, NodeControllerState> nodeControllerStateMap = ccs.getNodeMap();
+
+            Set<String> nodeIds = new TreeSet<String>();
+            for (String nc : nodeControllerStateMap.keySet()) {
+                nodeIds.add(nc);
+            }
+            final DeploymentRun dRun = new DeploymentRun(nodeIds);
+            ccs.addDeploymentRun(deploymentId, dRun);
+
+            /***
+             * deploy binaries to each node controller
+             */
+            for (NodeControllerState ncs : nodeControllerStateMap.values()) {
+                ncs.getNodeController().undeployBinary(deploymentId);
+            }
+
+            ccs.getExecutor().execute(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        /**
+                         * wait for completion
+                         */
+                        dRun.waitForCompletion();
+                        ccs.removeDeploymentRun(deploymentId);
+                        callback.setValue(null);
+                    } catch (Exception e) {
+                        callback.setException(e);
+                    }
+                }
+            });
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+}
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/GetResultStatusWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java
index d2dadf5..1bd0674 100644
--- 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
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
-import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.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;
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 b062d33..7ecdd16 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,8 +16,7 @@
 
 import java.util.EnumSet;
 
-import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGenerator;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
 import edu.uci.ics.hyracks.api.job.JobFlag;
@@ -26,6 +25,7 @@
 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;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentUtils;
 import edu.uci.ics.hyracks.control.common.work.IResultCallback;
 import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
@@ -33,31 +33,28 @@
     private final ClusterControllerService ccs;
     private final byte[] acggfBytes;
     private final EnumSet<JobFlag> jobFlags;
+    private final DeploymentId deploymentId;
     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, DeploymentId deploymentId, byte[] acggfBytes,
+            EnumSet<JobFlag> jobFlags, JobId jobId, IResultCallback<JobId> callback) {
+        this.deploymentId = deploymentId;
         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) DeploymentUtils
+                    .deserialize(acggfBytes, deploymentId, appCtx);
+            IActivityClusterGraphGenerator acgg = acggf.createActivityClusterGraphGenerator(jobId, appCtx, jobFlags);
+            JobRun run = new JobRun(ccs, deploymentId, 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/NotifyDeployBinaryWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NotifyDeployBinaryWork.java
new file mode 100644
index 0000000..903cad3
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NotifyDeployBinaryWork.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentRun;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentStatus;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class NotifyDeployBinaryWork extends AbstractWork {
+
+    private final ClusterControllerService ccs;
+    private final String nodeId;
+    private final DeploymentId deploymentId;
+    private DeploymentStatus deploymentStatus;
+
+    public NotifyDeployBinaryWork(ClusterControllerService ccs, DeploymentId deploymentId, String nodeId,
+            DeploymentStatus deploymentStatus) {
+        this.ccs = ccs;
+        this.nodeId = nodeId;
+        this.deploymentId = deploymentId;
+        this.deploymentStatus = deploymentStatus;
+
+    }
+
+    @Override
+    public void run() {
+        // notify that one NC is deployed
+        DeploymentRun dRun = ccs.getDeploymentRun(deploymentId);
+        dRun.notifyDeploymentStatus(nodeId, deploymentStatus);
+    }
+
+}
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/TaskFailureWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
index 0fff257..b3550bf 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
@@ -32,6 +32,7 @@
     @Override
     protected void performEvent(TaskAttempt ta) {
         JobRun run = ccs.getActiveRunMap().get(jobId);
+        //ccs.getDatasetDirectoryService().reportJobFailure(jobId);
         ActivityCluster ac = ta.getTask().getTaskCluster().getActivityCluster();
         run.getScheduler().notifyTaskFailure(ta, ac, details);
     }
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..58ae79e 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,23 @@
  */
 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.job.IJobSerializerDeserializerContainer;
+import edu.uci.ics.hyracks.api.job.JobSerializerDeserializerContainer;
 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;
+    protected IJobSerializerDeserializerContainer jobSerDeContainer = new JobSerializerDeserializerContainer();
 
-    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 +39,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;
     }
@@ -217,4 +47,9 @@
     public IMessageBroker getMessageBroker() {
         return this.messageBroker;
     }
+
+    @Override
+    public IJobSerializerDeserializerContainer getJobSerializerDeserializerContainer() {
+        return this.jobSerDeContainer;
+    }
 }
\ 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/IClusterController.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
index 55e4479..47a5c09 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
@@ -19,9 +19,10 @@
 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.deployment.DeploymentId;
 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.deployment.DeploymentStatus;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
 import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
 import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
@@ -40,6 +41,8 @@
 
     public void notifyJobletCleanup(JobId jobId, String nodeId) throws Exception;
 
+    public void notifyDeployBinary(DeploymentId deploymentId, String nodeId, DeploymentStatus status) throws Exception;
+
     public void nodeHeartbeat(String id, HeartbeatData hbData) throws Exception;
 
     public void reportProfile(String id, List<JobProfile> profiles) throws Exception;
@@ -48,6 +51,8 @@
 
     public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception;
 
+    public void sendApplicationMessageToCC(byte[] data, DeploymentId deploymentId, String nodeId) throws Exception;
+
     public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
             int nPartitions, NetworkAddress networkAddress) throws Exception;
 
@@ -55,9 +60,5 @@
 
     public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws Exception;
 
-    public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception;
-
-    public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception;
-
     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..cf27740 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
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.common.base;
 
+import java.net.URL;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
@@ -22,6 +23,7 @@
 import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -29,17 +31,17 @@
 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(DeploymentId deploymentId, 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;
+
+    public void deployBinary(DeploymentId deploymentId, List<URL> url) throws Exception;
+
+    public void undeployBinary(DeploymentId deploymentId) 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 5071bc9..ec29592 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
@@ -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.
@@ -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;
@@ -43,24 +45,25 @@
     @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 = "-result-history-size", usage = "Limits the number of jobs whose results should be remembered by the system to the specified value. (default: 10)")
+    public int resultHistorySize = 100;
+
     @Option(name = "-result-manager-memory", usage = "Memory usable for result caching at this Node Controller in bytes (default: -1 auto)")
     public int resultManagerMemory = -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;
+
     public void toCommandLine(List<String> cList) {
         cList.add("-cc-host");
         cList.add(ccHost);
@@ -75,19 +78,24 @@
         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));
+        cList.add("-result-history-size");
+        cList.add(String.valueOf(resultHistorySize));
         cList.add("-result-manager-memory");
         cList.add(String.valueOf(resultManagerMemory));
+
+        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);
+            }
+        }
     }
-}
\ 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/deployment/ClassLoaderJobSerializerDeserializer.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/ClassLoaderJobSerializerDeserializer.java
new file mode 100644
index 0000000..894f59b
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/ClassLoaderJobSerializerDeserializer.java
@@ -0,0 +1,81 @@
+package edu.uci.ics.hyracks.control.common.deployment;
+
+import java.io.Serializable;
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.IJobSerializerDeserializer;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
+
+public class ClassLoaderJobSerializerDeserializer implements IJobSerializerDeserializer {
+
+    private URLClassLoader classLoader;
+
+    @Override
+    public Object deserialize(byte[] jsBytes) throws HyracksException {
+        try {
+            if (classLoader == null) {
+                return JavaSerializationUtils.deserialize(jsBytes);
+            }
+            return JavaSerializationUtils.deserialize(jsBytes, classLoader);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    @Override
+    public byte[] serialize(Serializable jobSpec) throws HyracksException {
+        try {
+            if (classLoader == null) {
+                return JavaSerializationUtils.serialize(jobSpec);
+            }
+            return JavaSerializationUtils.serialize(jobSpec, classLoader);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    @Override
+    public void addClassPathURLs(List<URL> binaryURLs) throws HyracksException {
+        Collections.sort(binaryURLs, new Comparator<URL>() {
+            @Override
+            public int compare(URL o1, URL o2) {
+                return o1.getFile().compareTo(o2.getFile());
+            }
+        });
+        try {
+            if (classLoader == null) {
+                /** crate a new classloader */
+                URL[] urls = binaryURLs.toArray(new URL[binaryURLs.size()]);
+                classLoader = new URLClassLoader(urls, this.getClass().getClassLoader());
+            } else {
+                /** add URLs to the existing classloader */
+                Object[] urls = binaryURLs.toArray(new URL[binaryURLs.size()]);
+                Method method = classLoader.getClass().getDeclaredMethod("addURL", new Class[] { URL.class });
+                method.setAccessible(true);
+                method.invoke(classLoader, urls);
+            }
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    @Override
+    public Class<?> loadClass(String className) throws HyracksException {
+        try {
+            return classLoader.loadClass(className);
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    @Override
+    public ClassLoader getClassLoader() throws HyracksException {
+        return classLoader;
+    }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentRun.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentRun.java
new file mode 100644
index 0000000..519905c
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentRun.java
@@ -0,0 +1,29 @@
+package edu.uci.ics.hyracks.control.common.deployment;
+
+import java.util.Set;
+import java.util.TreeSet;
+
+public class DeploymentRun implements IDeploymentStatusConditionVariable {
+
+    private DeploymentStatus deploymentStatus = DeploymentStatus.FAIL;
+    private final Set<String> deploymentNodeIds = new TreeSet<String>();
+
+    public DeploymentRun(Set<String> nodeIds) {
+        deploymentNodeIds.addAll(nodeIds);
+    }
+
+    public synchronized void notifyDeploymentStatus(String nodeId, DeploymentStatus status) {
+        deploymentNodeIds.remove(nodeId);
+        if (deploymentNodeIds.size() == 0) {
+            deploymentStatus = DeploymentStatus.SUCCEED;
+            notifyAll();
+        }
+    }
+
+    @Override
+    public synchronized DeploymentStatus waitForCompletion() throws Exception {
+        wait();
+        return deploymentStatus;
+    }
+
+}
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentStatus.java
similarity index 78%
rename from hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
rename to hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentStatus.java
index 5f7ef0b..3690518 100644
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentStatus.java
@@ -12,11 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.cli;
 
-public class Main {
-    public static void main(String[] args) throws Exception {
-        CLI cli = new CLI(args);
-        cli.run();
-    }
-}
\ No newline at end of file
+package edu.uci.ics.hyracks.control.common.deployment;
+
+public enum DeploymentStatus {
+    SUCCEED,
+    FAIL
+}
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentUtils.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentUtils.java
new file mode 100644
index 0000000..715db96
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentUtils.java
@@ -0,0 +1,132 @@
+package edu.uci.ics.hyracks.control.common.deployment;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+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.application.IApplicationContext;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.IJobSerializerDeserializer;
+import edu.uci.ics.hyracks.api.job.IJobSerializerDeserializerContainer;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
+import edu.uci.ics.hyracks.control.common.context.ServerContext;
+
+public class DeploymentUtils {
+
+    private static final String DEPLOYMENT = "applications";
+
+    public static void undeploy(DeploymentId deploymentId, IJobSerializerDeserializerContainer container,
+            ServerContext ctx) throws HyracksException {
+        container.removeJobSerializerDeserializer(deploymentId);
+        String rootDir = ctx.getBaseDir().toString();
+        String deploymentDir = rootDir.endsWith(File.separator) ? rootDir + DEPLOYMENT + File.separator + deploymentId
+                : rootDir + File.separator + DEPLOYMENT + File.separator + deploymentId;
+        try {
+            File dFile = new File(deploymentDir);
+            if (dFile.exists()) {
+                FileUtils.forceDelete(dFile);
+            }
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    public static void deploy(DeploymentId deploymentId, List<URL> urls, IJobSerializerDeserializerContainer container,
+            ServerContext ctx, boolean isNC) throws HyracksException {
+        IJobSerializerDeserializer jobSerDe = container.getJobSerializerDeerializer(deploymentId);
+        if (jobSerDe == null) {
+            jobSerDe = new ClassLoaderJobSerializerDeserializer();
+            container.addJobSerializerDeserializer(deploymentId, jobSerDe);
+        }
+        String rootDir = ctx.getBaseDir().toString();
+        String deploymentDir = rootDir.endsWith(File.separator) ? rootDir + DEPLOYMENT + File.separator + deploymentId
+                : rootDir + File.separator + DEPLOYMENT + File.separator + deploymentId;
+        jobSerDe.addClassPathURLs(downloadURLs(urls, deploymentDir, isNC));
+    }
+
+    public static Object deserialize(byte[] bytes, DeploymentId deploymentId, IApplicationContext appCtx)
+            throws HyracksException {
+        try {
+            IJobSerializerDeserializerContainer jobSerDeContainer = appCtx.getJobSerializerDeserializerContainer();
+            IJobSerializerDeserializer jobSerDe = deploymentId == null ? null : jobSerDeContainer
+                    .getJobSerializerDeerializer(deploymentId);
+            Object obj = jobSerDe == null ? JavaSerializationUtils.deserialize(bytes) : jobSerDe.deserialize(bytes);
+            return obj;
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    public static Class<?> loadClass(String className, DeploymentId deploymentId, IApplicationContext appCtx)
+            throws HyracksException {
+        try {
+            IJobSerializerDeserializerContainer jobSerDeContainer = appCtx.getJobSerializerDeserializerContainer();
+            IJobSerializerDeserializer jobSerDe = deploymentId == null ? null : jobSerDeContainer
+                    .getJobSerializerDeerializer(deploymentId);
+            Class<?> cl = jobSerDe == null ? JavaSerializationUtils.loadClass(className) : jobSerDe
+                    .loadClass(className);
+            return cl;
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    public static ClassLoader getClassLoader(DeploymentId deploymentId, IApplicationContext appCtx)
+            throws HyracksException {
+        try {
+            IJobSerializerDeserializerContainer jobSerDeContainer = appCtx.getJobSerializerDeserializerContainer();
+            IJobSerializerDeserializer jobSerDe = deploymentId == null ? null : jobSerDeContainer
+                    .getJobSerializerDeerializer(deploymentId);
+            ClassLoader cl = jobSerDe == null ? DeploymentUtils.class.getClassLoader() : jobSerDe.getClassLoader();
+            return cl;
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+
+    private static List<URL> downloadURLs(List<URL> urls, String deploymentDir, boolean isNC) throws HyracksException {
+        try {
+            List<URL> downloadedFileURLs = new ArrayList<URL>();
+            File dir = new File(deploymentDir);
+            if (!dir.exists()) {
+                FileUtils.forceMkdir(dir);
+            }
+            for (URL url : urls) {
+                String urlString = url.toString();
+                int slashIndex = urlString.lastIndexOf('/');
+                String fileName = urlString.substring(slashIndex + 1).split("&")[1];
+                String filePath = deploymentDir + File.separator + fileName;
+                File targetFile = new File(filePath);
+                if (isNC) {
+                    HttpClient hc = new DefaultHttpClient();
+                    HttpGet get = new HttpGet(url.toString());
+                    HttpResponse response = hc.execute(get);
+                    InputStream is = response.getEntity().getContent();
+                    OutputStream os = new FileOutputStream(targetFile);
+                    try {
+                        IOUtils.copyLarge(is, os);
+                    } finally {
+                        os.close();
+                        is.close();
+                    }
+                }
+                downloadedFileURLs.add(targetFile.toURI().toURL());
+            }
+            return downloadedFileURLs;
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/IDeploymentStatusConditionVariable.java
similarity index 77%
copy from hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
copy to hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/IDeploymentStatusConditionVariable.java
index 5f7ef0b..06e398e 100644
--- a/hyracks/hyracks-cli/src/main/java/edu/uci/ics/hyracks/cli/Main.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/IDeploymentStatusConditionVariable.java
@@ -12,11 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.cli;
 
-public class Main {
-    public static void main(String[] args) throws Exception {
-        CLI cli = new CLI(args);
-        cli.run();
-    }
-}
\ No newline at end of file
+package edu.uci.ics.hyracks.control.common.deployment;
+
+public interface IDeploymentStatusConditionVariable {
+
+    public DeploymentStatus waitForCompletion() throws Exception;
+}
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 b506b12..e343657 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
@@ -21,6 +21,7 @@
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.Serializable;
+import java.net.URL;
 import java.nio.ByteBuffer;
 import java.util.EnumSet;
 import java.util.List;
@@ -37,13 +38,14 @@
 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.deployment.DeploymentId;
 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.deployment.DeploymentStatus;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
 import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
 import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
@@ -72,27 +74,32 @@
         REGISTER_RESULT_PARTITION_LOCATION,
         REPORT_RESULT_PARTITION_WRITE_COMPLETION,
         REPORT_RESULT_PARTITION_FAILURE,
-        APPLICATION_STATE_CHANGE_RESPONSE,
 
         NODE_REGISTRATION_RESULT,
         START_TASKS,
         ABORT_TASKS,
         CLEANUP_JOBLET,
-        CREATE_APPLICATION,
-        DESTROY_APPLICATION,
         REPORT_PARTITION_AVAILABILITY,
         SEND_APPLICATION_MESSAGE,
         GET_NODE_CONTROLLERS_INFO,
         GET_NODE_CONTROLLERS_INFO_RESPONSE,
 
+        DEPLOY_BINARY,
+        NOTIFY_DEPLOY_BINARY,
+        UNDEPLOY_BINARY,
+
         OTHER
     }
 
     public static class SendApplicationMessageFunction extends Function {
         private static final long serialVersionUID = 1L;
         private byte[] serializedMessage;
+        private DeploymentId deploymentId;
         private String nodeId;
-        private String appName;
+
+        public DeploymentId getDeploymentId() {
+            return deploymentId;
+        }
 
         public String getNodeId() {
             return nodeId;
@@ -106,11 +113,10 @@
             return serializedMessage;
         }
 
-        public SendApplicationMessageFunction(byte[] data, String appName, String nodeId) {
-            super();
+        public SendApplicationMessageFunction(byte[] data, DeploymentId deploymentId, String nodeId) {
             this.serializedMessage = data;
+            this.deploymentId = deploymentId;
             this.nodeId = nodeId;
-            this.appName = appName;
         }
 
         @Override
@@ -118,10 +124,6 @@
             return FunctionId.SEND_APPLICATION_MESSAGE;
         }
 
-        public String getAppName() {
-            return appName;
-        }
-
     }
 
     public static abstract class Function implements Serializable {
@@ -563,37 +565,6 @@
         }
     }
 
-    public static class ApplicationStateChangeResponseFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final String nodeId;
-        private final String appName;
-        private final ApplicationStatus status;
-
-        public ApplicationStateChangeResponseFunction(String nodeId, String appName, ApplicationStatus status) {
-            this.nodeId = nodeId;
-            this.appName = appName;
-            this.status = status;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.APPLICATION_STATE_CHANGE_RESPONSE;
-        }
-
-        public String getNodeId() {
-            return nodeId;
-        }
-
-        public String getApplicationName() {
-            return appName;
-        }
-
-        public ApplicationStatus getStatus() {
-            return status;
-        }
-    }
-
     public static class NodeRegistrationResult extends Function {
         private static final long serialVersionUID = 1L;
 
@@ -623,17 +594,17 @@
     public static class StartTasksFunction extends Function {
         private static final long serialVersionUID = 1L;
 
-        private final String appName;
+        private final DeploymentId deploymentId;
         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,
+        public StartTasksFunction(DeploymentId deploymentId, JobId jobId, byte[] planBytes,
                 List<TaskAttemptDescriptor> taskDescriptors,
                 Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, EnumSet<JobFlag> flags) {
-            this.appName = appName;
+            this.deploymentId = deploymentId;
             this.jobId = jobId;
             this.planBytes = planBytes;
             this.taskDescriptors = taskDescriptors;
@@ -646,8 +617,8 @@
             return FunctionId.START_TASKS;
         }
 
-        public String getAppName() {
-            return appName;
+        public DeploymentId getDeploymentId() {
+            return deploymentId;
         }
 
         public JobId getJobId() {
@@ -721,56 +692,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;
 
@@ -849,6 +770,81 @@
         }
     }
 
+    public static class DeployBinaryFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final List<URL> binaryURLs;
+        private final DeploymentId deploymentId;
+
+        public DeployBinaryFunction(DeploymentId deploymentId, List<URL> binaryURLs) {
+            this.binaryURLs = binaryURLs;
+            this.deploymentId = deploymentId;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.DEPLOY_BINARY;
+        }
+
+        public List<URL> getBinaryURLs() {
+            return binaryURLs;
+        }
+
+        public DeploymentId getDeploymentId() {
+            return deploymentId;
+        }
+    }
+
+    public static class UnDeployBinaryFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final DeploymentId deploymentId;
+
+        public UnDeployBinaryFunction(DeploymentId deploymentId) {
+            this.deploymentId = deploymentId;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.UNDEPLOY_BINARY;
+        }
+
+        public DeploymentId getDeploymentId() {
+            return deploymentId;
+        }
+    }
+
+    public static class NotifyDeployBinaryFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final String nodeId;
+        private final DeploymentId deploymentId;
+        private final DeploymentStatus deploymentStatus;
+
+        public NotifyDeployBinaryFunction(DeploymentId deploymentId, String nodeId, DeploymentStatus deploymentStatus) {
+            this.nodeId = nodeId;
+            this.deploymentId = deploymentId;
+            this.deploymentStatus = deploymentStatus;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.NOTIFY_DEPLOY_BINARY;
+        }
+
+        public String getNodeId() {
+            return nodeId;
+        }
+
+        public DeploymentId getDeploymentId() {
+            return deploymentId;
+        }
+
+        public DeploymentStatus getDeploymentStatus() {
+            return deploymentStatus;
+        }
+    }
+
     public static class SerializerDeserializer implements IPayloadSerializerDeserializer {
         private final JavaSerializationBasedPayloadSerializerDeserializer javaSerde;
 
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 091a5d2..5ed65cc 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
@@ -19,10 +19,11 @@
 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.deployment.DeploymentId;
 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.deployment.DeploymentStatus;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
 import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
 import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
@@ -71,6 +72,13 @@
     }
 
     @Override
+    public void notifyDeployBinary(DeploymentId deploymentId, String nodeId, DeploymentStatus status) throws Exception {
+        CCNCFunctions.NotifyDeployBinaryFunction fn = new CCNCFunctions.NotifyDeployBinaryFunction(deploymentId,
+                nodeId, status);
+        ipcHandle.send(-1, fn, null);
+    }
+
+    @Override
     public void nodeHeartbeat(String id, HeartbeatData hbData) throws Exception {
         CCNCFunctions.NodeHeartbeatFunction fn = new CCNCFunctions.NodeHeartbeatFunction(id, hbData);
         ipcHandle.send(-1, fn, null);
@@ -97,6 +105,12 @@
     }
 
     @Override
+    public void sendApplicationMessageToCC(byte[] data, DeploymentId deploymentId, String nodeId) throws Exception {
+        CCNCFunctions.SendApplicationMessageFunction fn = new CCNCFunctions.SendApplicationMessageFunction(data,
+                deploymentId, 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(
@@ -119,21 +133,8 @@
     }
 
     @Override
-    public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception {
-        CCNCFunctions.ApplicationStateChangeResponseFunction fn = new CCNCFunctions.ApplicationStateChangeResponseFunction(
-                nodeId, appName, status);
-        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);
-        ipcHandle.send(-1, fn, null);
-    }
-
-    @Override
     public void getNodeControllerInfos() throws Exception {
         ipcHandle.send(-1, new CCNCFunctions.GetNodeControllersInfoFunction(), null);
     }
+
 }
\ No newline at end of file
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..8346ecb 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
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.common.ipc;
 
+import java.net.URL;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
@@ -22,6 +23,7 @@
 import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -38,9 +40,10 @@
     }
 
     @Override
-    public void startTasks(String appName, JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
+    public void startTasks(DeploymentId deploymentId, 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,
+        CCNCFunctions.StartTasksFunction stf = new CCNCFunctions.StartTasksFunction(deploymentId, jobId, planBytes,
                 taskDescriptors, connectorPolicies, flags);
         ipcHandle.send(-1, stf, null);
     }
@@ -58,23 +61,21 @@
     }
 
     @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);
         ipcHandle.send(-1, rpaf, null);
     }
+
+    @Override
+    public void deployBinary(DeploymentId deploymentId, List<URL> binaryURLs) throws Exception {
+        CCNCFunctions.DeployBinaryFunction rpaf = new CCNCFunctions.DeployBinaryFunction(deploymentId, binaryURLs);
+        ipcHandle.send(-1, rpaf, null);
+    }
+
+    @Override
+    public void undeployBinary(DeploymentId deploymentId) throws Exception {
+        CCNCFunctions.UnDeployBinaryFunction rpaf = new CCNCFunctions.UnDeployBinaryFunction(deploymentId);
+        ipcHandle.send(-1, rpaf, null);
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/pom.xml b/hyracks/hyracks-control/hyracks-control-nc/pom.xml
index b6eff7a..df5c998 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.4-SNAPSHOT</version>
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index 3855b4d..c6a03dc 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -27,6 +27,7 @@
 import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -43,6 +44,7 @@
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
 import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentUtils;
 import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
 import edu.uci.ics.hyracks.control.common.job.PartitionState;
 import edu.uci.ics.hyracks.control.common.job.profiling.counters.Counter;
@@ -58,6 +60,8 @@
 
     private final INCApplicationContext appCtx;
 
+    private final DeploymentId deploymentId;
+
     private final JobId jobId;
 
     private final ActivityClusterGraph acg;
@@ -86,10 +90,11 @@
 
     private boolean cleanupPending;
 
-    public Joblet(NodeControllerService nodeController, JobId jobId, INCApplicationContext appCtx,
-            ActivityClusterGraph acg) {
+    public Joblet(NodeControllerService nodeController, DeploymentId deploymentId, JobId jobId,
+            INCApplicationContext appCtx, ActivityClusterGraph acg) {
         this.nodeController = nodeController;
         this.appCtx = appCtx;
+        this.deploymentId = deploymentId;
         this.jobId = jobId;
         this.frameSize = acg.getFrameSize();
         this.acg = acg;
@@ -283,4 +288,22 @@
             e.printStackTrace();
         }
     }
+
+    @Override
+    public Class<?> loadClass(String className) {
+        try {
+            return DeploymentUtils.loadClass(className, deploymentId, appCtx);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public ClassLoader getClassLoader() {
+        try {
+            return DeploymentUtils.getClassLoader(deploymentId, appCtx);
+        } 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/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 8a36dac..a1c3b08 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,9 +43,11 @@
 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.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
@@ -72,10 +74,10 @@
 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.DeployBinaryWork;
 import edu.uci.ics.hyracks.control.nc.work.ReportPartitionAvailabilityWork;
 import edu.uci.ics.hyracks.control.nc.work.StartTasksWork;
+import edu.uci.ics.hyracks.control.nc.work.UnDeployBinaryWork;
 import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
 import edu.uci.ics.hyracks.ipc.api.IIPCI;
 import edu.uci.ics.hyracks.ipc.api.IPCPerformanceCounters;
@@ -121,7 +123,9 @@
 
     private final ServerContext serverCtx;
 
-    private final Map<String, NCApplicationContext> applications;
+    private NCApplicationContext appCtx;
+
+    private INCApplicationEntryPoint ncAppEntryPoint;
 
     private final MemoryMXBean memoryMXBean;
 
@@ -149,7 +153,8 @@
         partitionManager = new PartitionManager(this);
         netManager = new NetworkManager(getIpAddress(ncConfig.dataIPAddress), partitionManager, ncConfig.nNetThreads);
 
-        datasetPartitionManager = new DatasetPartitionManager(this, executor, ncConfig.resultManagerMemory);
+        datasetPartitionManager = new DatasetPartitionManager(this, executor, ncConfig.resultManagerMemory,
+                ncConfig.resultHistorySize);
         datasetNetworkManager = new DatasetNetworkManager(getIpAddress(ncConfig.datasetIPAddress),
                 datasetPartitionManager, ncConfig.nNetThreads);
 
@@ -158,7 +163,6 @@
         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();
@@ -172,6 +176,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, ",");
@@ -216,6 +224,9 @@
         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);
@@ -239,6 +250,7 @@
         if (registrationException != null) {
             throw registrationException;
         }
+        appCtx.setDistributedState(nodeParameters.getDistributedState());
 
         queue.start();
 
@@ -253,6 +265,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
@@ -276,10 +303,6 @@
         return serverCtx;
     }
 
-    public Map<String, NCApplicationContext> getApplications() {
-        return applications;
-    }
-
     public Map<JobId, Joblet> getJobletMap() {
         return jobletMap;
     }
@@ -424,13 +447,14 @@
                 case SEND_APPLICATION_MESSAGE: {
                     CCNCFunctions.SendApplicationMessageFunction amf = (CCNCFunctions.SendApplicationMessageFunction) fn;
                     queue.schedule(new ApplicationMessageWork(NodeControllerService.this, amf.getMessage(), amf
-                            .getAppName(), amf.getNodeId()));
+                            .getDeploymentId(), amf.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.getDeploymentId(),
+                            stf.getJobId(), stf.getPlanBytes(), stf.getTaskDescriptors(), stf.getConnectorPolicies(),
+                            stf.getFlags()));
                     return;
                 }
 
@@ -446,19 +470,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
@@ -477,17 +488,30 @@
                     setNodeControllersInfo(gncirf.getNodeControllerInfos());
                     return;
                 }
+
+                case DEPLOY_BINARY: {
+                    CCNCFunctions.DeployBinaryFunction ndbf = (CCNCFunctions.DeployBinaryFunction) fn;
+                    queue.schedule(new DeployBinaryWork(NodeControllerService.this, ndbf.getDeploymentId(), ndbf
+                            .getBinaryURLs()));
+                    return;
+                }
+
+                case UNDEPLOY_BINARY: {
+                    CCNCFunctions.UnDeployBinaryFunction ndbf = (CCNCFunctions.UnDeployBinaryFunction) fn;
+                    queue.schedule(new UnDeployBinaryWork(NodeControllerService.this, ndbf.getDeploymentId()));
+                    return;
+                }
             }
             throw new IllegalArgumentException("Unknown function: " + fn.getFunctionId());
 
         }
     }
 
-    public IDatasetPartitionManager getDatasetPartitionManager() {
-        return datasetPartitionManager;
+    public void sendApplicationMessageToCC(byte[] data, DeploymentId deploymentId, String nodeId) throws Exception {
+        ccs.sendApplicationMessageToCC(data, deploymentId, nodeId);
     }
 
-    public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception {
-        ccs.sendApplicationMessageToCC(data, appName, 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 5a3e9dd..ac76c16 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
@@ -35,6 +35,7 @@
 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.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -354,8 +355,7 @@
     }
 
     @Override
-    public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception {
-        this.ncs.sendApplicationMessageToCC(message, this.getJobletContext().getApplicationContext()
-                .getApplicationName(), nodeId);
+    public void sendApplicationMessageToCC(byte[] message, DeploymentId deploymentId, String nodeId) throws Exception {
+        this.ncs.sendApplicationMessageToCC(message, deploymentId, 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/DatasetPartitionManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java
index 1cad54b..1e58b5c 100644
--- 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
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.nc.dataset;
 
-import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.concurrent.Executor;
 
@@ -44,13 +44,26 @@
 
     private final DatasetMemoryManager datasetMemoryManager;
 
-    public DatasetPartitionManager(NodeControllerService ncs, Executor executor, int availableMemory) {
+    public DatasetPartitionManager(NodeControllerService ncs, Executor executor, int availableMemory,
+            final int resultHistorySize) {
         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);
+        partitionResultStateMap = new LinkedHashMap<JobId, ResultState[]>() {
+            private static final long serialVersionUID = 1L;
+
+            protected boolean removeEldestEntry(Map.Entry<JobId, ResultState[]> eldest) {
+                if (size() > resultHistorySize) {
+                    for (ResultState state : eldest.getValue()) {
+                        state.deinit();
+                    }
+                    return true;
+                }
+                return false;
+            }
+        };
     }
 
     @Override
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
index 3db3fd9..0f1d94c 100644
--- 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
@@ -64,6 +64,10 @@
         notifyAll();
     }
 
+    public synchronized void deinit() {
+        fileRef.delete();
+    }
+
     public ResultSetPartitionId getResultSetPartitionId() {
         return resultSetPartitionId;
     }
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..19a5a81 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
@@ -14,49 +14,46 @@
  */
 package edu.uci.ics.hyracks.control.nc.work;
 
-import java.io.IOException;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.messages.IMessage;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentUtils;
 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 DeploymentId deploymentId;
     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, DeploymentId deploymentId, String nodeId) {
         this.ncs = ncs;
+        this.deploymentId = deploymentId;
         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) DeploymentUtils.deserialize(message, deploymentId, ctx);;
             if (ctx.getMessageBroker() != null) {
                 ctx.getMessageBroker().receivedMessage(data, nodeId);
             } else {
                 LOGGER.log(Level.WARNING, "Messsage was sent, but no Message Broker set!");
             }
-        } catch (IOException e) {
+        } catch (Exception e) {
             Logger.getLogger(this.getClass().getName()).log(Level.WARNING, "Error in application message delivery!", e);
-        } catch (ClassNotFoundException e) {
-            Logger.getLogger(this.getClass().getName()).log(Level.WARNING, "Error in application message delivery!", e);
+            throw new RuntimeException(e);
         }
     }
 
@@ -64,5 +61,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/DeployBinaryWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DeployBinaryWork.java
new file mode 100644
index 0000000..4f613a1
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DeployBinaryWork.java
@@ -0,0 +1,37 @@
+package edu.uci.ics.hyracks.control.nc.work;
+
+import java.net.URL;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
+import edu.uci.ics.hyracks.control.common.base.IClusterController;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentStatus;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentUtils;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+
+public class DeployBinaryWork extends AbstractWork {
+
+    private DeploymentId deploymentId;
+    private NodeControllerService ncs;
+    private List<URL> binaryURLs;
+
+    public DeployBinaryWork(NodeControllerService ncs, DeploymentId deploymentId, List<URL> binaryURLs) {
+        this.deploymentId = deploymentId;
+        this.ncs = ncs;
+        this.binaryURLs = binaryURLs;
+    }
+
+    @Override
+    public void run() {
+        try {
+            DeploymentUtils.deploy(deploymentId, binaryURLs, ncs.getApplicationContext()
+                    .getJobSerializerDeserializerContainer(), ncs.getServerContext(), true);
+            IClusterController ccs = ncs.getClusterController();
+            ccs.notifyDeployBinary(deploymentId, ncs.getId(), DeploymentStatus.SUCCEED);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+}
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/StartTasksWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
index 0c0fa3d..2c6d38e 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
@@ -36,12 +36,14 @@
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.ActivityCluster;
 import edu.uci.ics.hyracks.api.job.ActivityClusterGraph;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentUtils;
 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,7 +61,7 @@
 
     private final NodeControllerService ncs;
 
-    private final String appName;
+    private final DeploymentId deploymentId;
 
     private final JobId jobId;
 
@@ -71,11 +73,11 @@
 
     private final EnumSet<JobFlag> flags;
 
-    public StartTasksWork(NodeControllerService ncs, String appName, JobId jobId, byte[] acgBytes,
+    public StartTasksWork(NodeControllerService ncs, DeploymentId deploymentId, JobId jobId, byte[] acgBytes,
             List<TaskAttemptDescriptor> taskDescriptors,
             Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap, EnumSet<JobFlag> flags) {
         this.ncs = ncs;
-        this.appName = appName;
+        this.deploymentId = deploymentId;
         this.jobId = jobId;
         this.acgBytes = acgBytes;
         this.taskDescriptors = taskDescriptors;
@@ -86,10 +88,9 @@
     @Override
     public void run() {
         try {
-            Map<String, NCApplicationContext> applications = ncs.getApplications();
-            NCApplicationContext appCtx = applications.get(appName);
-            final Joblet joblet = getOrCreateLocalJoblet(jobId, appCtx, acgBytes == null ? null
-                    : (ActivityClusterGraph) appCtx.deserialize(acgBytes));
+            NCApplicationContext appCtx = ncs.getApplicationContext();
+            final Joblet joblet = getOrCreateLocalJoblet(deploymentId, jobId, appCtx, acgBytes == null ? null
+                    : (ActivityClusterGraph) DeploymentUtils.deserialize(acgBytes, deploymentId, appCtx));
             final ActivityClusterGraph acg = joblet.getActivityClusterGraph();
 
             IRecordDescriptorProvider rdp = new IRecordDescriptorProvider() {
@@ -167,15 +168,15 @@
         }
     }
 
-    private Joblet getOrCreateLocalJoblet(JobId jobId, INCApplicationContext appCtx, ActivityClusterGraph acg)
-            throws Exception {
+    private Joblet getOrCreateLocalJoblet(DeploymentId deploymentId, JobId jobId, INCApplicationContext appCtx,
+            ActivityClusterGraph acg) throws Exception {
         Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
         Joblet ji = jobletMap.get(jobId);
         if (ji == null) {
             if (acg == null) {
                 throw new NullPointerException("JobActivityGraph was null");
             }
-            ji = new Joblet(ncs, jobId, appCtx, acg);
+            ji = new Joblet(ncs, deploymentId, jobId, appCtx, acg);
             jobletMap.put(jobId, ji);
         }
         return ji;
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/UnDeployBinaryWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/UnDeployBinaryWork.java
new file mode 100644
index 0000000..bd77fb5
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/UnDeployBinaryWork.java
@@ -0,0 +1,32 @@
+package edu.uci.ics.hyracks.control.nc.work;
+
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
+import edu.uci.ics.hyracks.control.common.base.IClusterController;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentStatus;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentUtils;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+
+public class UnDeployBinaryWork extends AbstractWork {
+
+    private DeploymentId deploymentId;
+    private NodeControllerService ncs;
+
+    public UnDeployBinaryWork(NodeControllerService ncs, DeploymentId deploymentId) {
+        this.deploymentId = deploymentId;
+        this.ncs = ncs;
+    }
+
+    @Override
+    public void run() {
+        try {
+            DeploymentUtils.undeploy(deploymentId, ncs.getApplicationContext().getJobSerializerDeserializerContainer(),
+                    ncs.getServerContext());
+            IClusterController ccs = ncs.getClusterController();
+            ccs.notifyDeployBinary(deploymentId, ncs.getId(), DeploymentStatus.SUCCEED);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+}
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 f392b34..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.4-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.4-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 b6e8c72..8785f16 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
@@ -62,9 +62,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;
 
@@ -88,7 +85,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 a6c7ea6..e9c3b1c 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
@@ -60,9 +60,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;
 
@@ -86,7 +83,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 d24ba33..f9e567c 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
@@ -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;
 
@@ -79,7 +76,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 5aa338a..bb3e174 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
@@ -55,9 +55,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/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 277668b..1b4bb40 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
@@ -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;
 
@@ -82,7 +79,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 304c8ce..93b5f4f 100644
--- a/hyracks/hyracks-cli/pom.xml
+++ b/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
@@ -1,45 +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>
-  <groupId>edu.uci.ics.hyracks</groupId>
-  <artifactId>hyracks-cli</artifactId>
+  <groupId>edu.uci.ics.hyracks.examples.btree</groupId>
+  <artifactId>btreeserver</artifactId>
   <version>0.2.4-SNAPSHOT</version>
-  <name>hyracks-cli</name>
+  <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.4-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>
@@ -48,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>
@@ -83,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.4-SNAPSHOT</version>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-cc</artifactId>
+  		<version>0.2.4-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.4-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 b6fbb95..135665a 100644
--- a/hyracks/hyracks-examples/btree-example/pom.xml
+++ b/hyracks/hyracks-examples/btree-example/pom.xml
@@ -15,6 +15,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 61%
rename from hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
rename to hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatserver/pom.xml
index 6171214..2eb35e1 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.4-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.4-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,29 +95,16 @@
               <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>
@@ -174,6 +139,20 @@
         <version>0.2.4-SNAPSHOT</version>
         <scope>compile</scope>
      </dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-cc</artifactId>
+  		<version>0.2.4-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-nc</artifactId>
+  		<version>0.2.4-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 9d07031..b4ba2a3 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/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 8893567..acaa94a 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
@@ -106,7 +106,6 @@
         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());
         }
@@ -123,7 +122,7 @@
         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());
         }
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 24d0ef4..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
@@ -100,7 +100,6 @@
         }
 
         hcc = new HyracksConnection(ccConfig.clientNetIpAddress, ccConfig.clientNetPort);
-        hcc.createApplication("test", null);
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting CC in " + ccRoot.getAbsolutePath());
         }
@@ -118,7 +117,7 @@
         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());
         }
diff --git a/hyracks/hyracks-examples/text-example/pom.xml b/hyracks/hyracks-examples/text-example/pom.xml
index a183835..a4b0311 100644
--- a/hyracks/hyracks-examples/text-example/pom.xml
+++ b/hyracks/hyracks-examples/text-example/pom.xml
@@ -14,6 +14,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 65%
rename from hyracks/hyracks-examples/text-example/textapp/pom.xml
rename to hyracks/hyracks-examples/text-example/textserver/pom.xml
index 5927e83..d3eaaca 100644
--- a/hyracks/hyracks-examples/text-example/textapp/pom.xml
+++ b/hyracks/hyracks-examples/text-example/textserver/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.text</groupId>
-  <artifactId>textapp</artifactId>
-  <name>textapp</name>
+  <artifactId>textserver</artifactId>
+  <version>0.2.4-SNAPSHOT</version>
+  <name>textserver</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.4-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>
@@ -116,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>
@@ -170,6 +140,20 @@
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-cc</artifactId>
+  		<version>0.2.4-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-nc</artifactId>
+  		<version>0.2.4-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
   		<groupId>edu.uci.ics.hyracks.examples.text</groupId>
   		<artifactId>textclient</artifactId>
   		<version>0.2.4-SNAPSHOT</version>
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 a36efc9..5389ca3 100644
--- a/hyracks/hyracks-examples/tpch-example/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/pom.xml
@@ -13,6 +13,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 89689c2..0000000
--- a/hyracks/hyracks-examples/tpch-example/tpchapp/pom.xml
+++ /dev/null
@@ -1,91 +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.tpch</groupId>
-  <artifactId>tpchapp</artifactId>
-  <name>tpchapp</name>
-  <parent>
-    <groupId>edu.uci.ics.hyracks.examples</groupId>
-    <artifactId>tpch-example</artifactId>
-    <version>0.2.4-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.4-SNAPSHOT</version>
-        <scope>compile</scope>
-    </dependency>
-    <dependency>
-    	<groupId>edu.uci.ics.hyracks</groupId>
-    	<artifactId>hyracks-data-std</artifactId>
-    	<version>0.2.4-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 304c8ce..0110fc4 100644
--- a/hyracks/hyracks-cli/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
@@ -1,45 +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>
-  <groupId>edu.uci.ics.hyracks</groupId>
-  <artifactId>hyracks-cli</artifactId>
+  <groupId>edu.uci.ics.hyracks.examples.tpch</groupId>
+  <artifactId>tpchserver</artifactId>
   <version>0.2.4-SNAPSHOT</version>
-  <name>hyracks-cli</name>
+  <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.4-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>
@@ -48,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>
@@ -82,17 +60,29 @@
     </plugins>
   </build>
   <dependencies>
+    <dependency>
+        <groupId>edu.uci.ics.hyracks</groupId>
+        <artifactId>hyracks-dataflow-std</artifactId>
+        <version>0.2.4-SNAPSHOT</version>
+        <scope>compile</scope>
+    </dependency>
+    <dependency>
+    	<groupId>edu.uci.ics.hyracks</groupId>
+    	<artifactId>hyracks-data-std</artifactId>
+    	<version>0.2.4-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.4-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.4-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/pom.xml b/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml
index b693c0a..2a4b0d1 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml
@@ -52,18 +52,6 @@
 					</filesets>
 				</configuration>
 			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<version>2.2</version>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
 		</plugins>
 	</build>
 
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 f3ec2fd..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
@@ -39,63 +39,55 @@
 import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
 
 /**
- * The HDFS file write operator using the Hadoop old API. To use this operator,
- * a user need to provide an ITupleWriterFactory.
+ * The HDFS file write operator using the Hadoop old API.
+ * To use this operator, a user need to provide an ITupleWriterFactory.
  */
 @SuppressWarnings("deprecation")
-public class HDFSWriteOperatorDescriptor extends
-		AbstractSingleActivityOperatorDescriptor {
+public class HDFSWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
 
-	private static final long serialVersionUID = 1L;
-	private ConfFactory confFactory;
-	private ITupleWriterFactory tupleWriterFactory;
+    private static final long serialVersionUID = 1L;
+    private ConfFactory confFactory;
+    private ITupleWriterFactory tupleWriterFactory;
 
-	/**
-	 * The constructor of HDFSWriteOperatorDescriptor.
-	 * 
-	 * @param spec
-	 *            the JobSpecification object
-	 * @param conf
-	 *            the Hadoop JobConf which contains the output path
-	 * @param tupleWriterFactory
-	 *            the ITupleWriterFactory implementation object
-	 * @throws HyracksException
-	 */
-	public HDFSWriteOperatorDescriptor(JobSpecification spec, JobConf conf,
-			ITupleWriterFactory tupleWriterFactory) throws HyracksException {
-		super(spec, 1, 0);
-		this.confFactory = new ConfFactory(conf);
-		this.tupleWriterFactory = tupleWriterFactory;
-	}
+    /**
+     * The constructor of HDFSWriteOperatorDescriptor.
+     * 
+     * @param spec
+     *            the JobSpecification object
+     * @param conf
+     *            the Hadoop JobConf which contains the output path
+     * @param tupleWriterFactory
+     *            the ITupleWriterFactory implementation object
+     * @throws HyracksException
+     */
+    public HDFSWriteOperatorDescriptor(JobSpecification spec, JobConf conf, ITupleWriterFactory tupleWriterFactory)
+            throws HyracksException {
+        super(spec, 1, 0);
+        this.confFactory = new ConfFactory(conf);
+        this.tupleWriterFactory = tupleWriterFactory;
+    }
 
-	@Override
-	public IOperatorNodePushable createPushRuntime(
-			final IHyracksTaskContext ctx,
-			final IRecordDescriptorProvider recordDescProvider,
-			final int partition, final int nPartitions)
-			throws HyracksDataException {
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+            throws HyracksDataException {
 
-		return new AbstractUnaryInputSinkOperatorNodePushable() {
+        return new AbstractUnaryInputSinkOperatorNodePushable() {
 
-			private FSDataOutputStream dos;
-			private RecordDescriptor inputRd = recordDescProvider
-					.getInputRecordDescriptor(getActivityId(), 0);;
-			private FrameTupleAccessor accessor = new FrameTupleAccessor(
-					ctx.getFrameSize(), inputRd);
-			private FrameTupleReference tuple = new FrameTupleReference();
-			private ITupleWriter tupleWriter;
-			private ClassLoader ctxCL;
+            private FSDataOutputStream dos;
+            private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
+            private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRd);
+            private FrameTupleReference tuple = new FrameTupleReference();
+            private ITupleWriter tupleWriter;
+            private ClassLoader ctxCL;
 
-			@Override
-			public void open() throws HyracksDataException {
-				ctxCL = Thread.currentThread().getContextClassLoader();
-				Thread.currentThread().setContextClassLoader(
-						this.getClass().getClassLoader());
-				JobConf conf = confFactory.getConf();
-				String outputDirPath = FileOutputFormat.getOutputPath(conf)
-						.toString();
-				String fileName = outputDirPath + File.separator + "part-"
-						+ partition;
+            @Override
+            public void open() throws HyracksDataException {
+                ctxCL = Thread.currentThread().getContextClassLoader();
+                Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+                JobConf conf = confFactory.getConf();
+                String outputDirPath = FileOutputFormat.getOutputPath(conf).toString();
+                String fileName = outputDirPath + File.separator + "part-" + partition;
 
                 tupleWriter = tupleWriterFactory.getTupleWriter(ctx);
                 try {
@@ -107,21 +99,20 @@
                 }
             }
 
-			@Override
-			public void nextFrame(ByteBuffer buffer)
-					throws HyracksDataException {
-				accessor.reset(buffer);
-				int tupleCount = accessor.getTupleCount();
-				for (int i = 0; i < tupleCount; i++) {
-					tuple.reset(accessor, i);
-					tupleWriter.write(dos, tuple);
-				}
-			}
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                accessor.reset(buffer);
+                int tupleCount = accessor.getTupleCount();
+                for (int i = 0; i < tupleCount; i++) {
+                    tuple.reset(accessor, i);
+                    tupleWriter.write(dos, tuple);
+                }
+            }
 
-			@Override
-			public void fail() throws HyracksDataException {
+            @Override
+            public void fail() throws HyracksDataException {
 
-			}
+            }
 
             @Override
             public void close() throws HyracksDataException {
@@ -135,6 +126,6 @@
                 }
             }
 
-		};
-	}
+        };
+    }
 }
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java
index 9e9abdf..8b58ecd 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java
@@ -114,6 +114,7 @@
                 try {
                     Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
                     Job job = confFactory.getConf();
+                    job.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
                     IKeyValueParser parser = tupleParserFactory.createKeyValueParser(ctx);
                     writer.open();
                     InputFormat inputFormat = ReflectionUtils.newInstance(job.getInputFormatClass(),
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/utils/HyracksUtils.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
index 8c12518..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
@@ -83,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();
@@ -99,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/hdfs/utils/TestUtils.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/TestUtils.java
index e155a2c..3826688 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/TestUtils.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/TestUtils.java
@@ -15,12 +15,9 @@
 
 package edu.uci.ics.hyracks.hdfs.utils;
 
-import java.awt.List;
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileReader;
-import java.util.ArrayList;
-import java.util.Collections;
 
 public class TestUtils {
 
@@ -51,44 +48,6 @@
             readerActual.close();
         }
     }
-    
-    /**
-     * Compare with the sorted expected file.
-     * The actual file may not be sorted; 
-     * @param expectedFile
-     * @param actualFile
-     */
-    public static void compareWithSortedResult(File expectedFile, File actualFile) throws Exception{
-    	BufferedReader readerActual = new BufferedReader(new FileReader(actualFile));
-    	BufferedReader readerExpected = new BufferedReader(new FileReader(expectedFile));
-    	ArrayList<String> actualLines = new ArrayList<String>();
-    	String lineExpected, lineActual;
-    	try{
-    		while ( (lineActual = readerActual.readLine())!=null){
-    			actualLines.add(lineActual);
-    		}
-    		Collections.sort(actualLines);
-    		int num = 1;
-    		for(String actualLine : actualLines){
-    			lineExpected = readerExpected.readLine();
-    			if (lineExpected == null){
-    				throw new Exception("Actual result changed at line " + num + ":\n< " + actualLine + "\n> ");
-    			}
-    			if ( !equalStrings(lineExpected, actualLine)){
-    				   throw new Exception("Result for changed at line " + num + ":\n< " + lineExpected + "\n> "
-                               + actualLine);
-    			}
-                ++num;
-    		}
-	        lineExpected = readerExpected.readLine();
-	        if (lineExpected != null) {
-	            throw new Exception("Actual result changed at line " + num + ":\n< \n> " + lineExpected);
-	        }
-    	} finally{
-    		readerActual.close();
-    		readerExpected.close();
-    	}
-    }
 
     private static boolean equalStrings(String s1, String s2) {
         String[] rowsOne = s1.split("\n");
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-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-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
index 7612db9..8285efe 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
@@ -92,4 +92,18 @@
     public Object getGlobalJobData() {
         return null;
     }
+
+    @Override
+    public Class<?> loadClass(String className) {
+        try {
+            return Class.forName(className);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public ClassLoader getClassLoader() {
+        return this.getClass().getClassLoader();
+    }
 }
\ No newline at end of file
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..285ab1f 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
@@ -18,6 +18,7 @@
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.job.IJobSerializerDeserializerContainer;
 import edu.uci.ics.hyracks.api.messages.IMessageBroker;
 
 public class TestNCApplicationContext implements INCApplicationContext {
@@ -38,11 +39,6 @@
     }
 
     @Override
-    public ClassLoader getClassLoader() {
-        return getClass().getClassLoader();
-    }
-
-    @Override
     public Serializable getDistributedState() {
         return distributedState;
     }
@@ -63,15 +59,9 @@
     }
 
     @Override
-    public String getApplicationName() {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
     public void setMessageBroker(IMessageBroker staticticsConnector) {
         // TODO Auto-generated method stub
-        
+
     }
 
     @Override
@@ -79,4 +69,10 @@
         // TODO Auto-generated method stub
         return null;
     }
+
+	@Override
+	public IJobSerializerDeserializerContainer getJobSerializerDeserializerContainer() {
+		// TODO Auto-generated method stub
+		return null;
+	}
 }
\ No newline at end of file
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 0ca93b2..8311ebd 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
@@ -21,6 +21,7 @@
 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.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -107,7 +108,7 @@
     }
 
     @Override
-    public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception {
+    public void sendApplicationMessageToCC(byte[] message, DeploymentId deploymentId, 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 2810aa1..6c84eb8 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}</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>
@@ -91,7 +81,6 @@
     <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-btree</module>
diff --git a/pom.xml b/pom.xml
index 1042f6e..2840cb1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -68,6 +68,5 @@
     <module>algebricks</module>
     <module>pregelix</module>
     <module>hivesterix</module>
-    <module>genomix</module>
   </modules>
 </project>
diff --git a/pregelix/pom.xml b/pregelix/pom.xml
index 0d2bdba..5c01e31 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 b8bfce9..85f6ea2 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 724ec14..e51d4bc 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,531 +49,524 @@
  */
 @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);
-		updated = true;
-		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;
-		updated = false;
-	}
+    /**
+     * 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;
-		updated = 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;
-	}
+    /**
+     * Activate a halted vertex such that it is alive again.
+     */
+    public final void activate() {
+        halt = false;
+        updated = true;
+    }
 
-	@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();
+    /**
+     * @return the vertex is halted (true) or not (false)
+     */
+    public final boolean isHalted() {
+        return halt;
+    }
 
-		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
+    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
-	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);
-	}
+        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;
+    }
 
-	/**
-	 * Get the list of incoming messages
-	 * 
-	 * @return the list of messages
-	 */
-	public List<M> getMsgList() {
-		return msgList;
-	}
+    @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 outgoing edge list
-	 * 
-	 * @return a list of outgoing edges
-	 */
-	public List<Edge<I, E>> getEdges() {
-		return this.destEdgeList;
-	}
+    /**
+     * Get the list of incoming messages
+     * 
+     * @return the list of messages
+     */
+    public List<M> getMsgList() {
+        return msgList;
+    }
 
-	@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 outgoing edge list
+     * 
+     * @return a list of outgoing edges
+     */
+    public List<Edge<I, E>> getEdges() {
+        return this.destEdgeList;
+    }
 
-	/**
-	 * Get the number of outgoing edges
-	 * 
-	 * @return the number of outging edges
-	 */
-	public int getNumOutEdges() {
-		return destEdgeList.size();
-	}
+    @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 + ")";
+    }
 
-	/**
-	 * Pregelix internal use only
-	 * 
-	 * @param writers
-	 */
-	public void setOutputWriters(List<IFrameWriter> writers) {
-		delegate.setOutputWriters(writers);
-	}
+    /**
+     * 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 setOutputAppenders(List<FrameTupleAppender> appenders) {
-		delegate.setOutputAppenders(appenders);
-	}
+    /**
+     * Pregelix internal use only
+     * 
+     * @param writers
+     */
+    public void setOutputWriters(List<IFrameWriter> writers) {
+        delegate.setOutputWriters(writers);
+    }
 
-	/**
-	 * Pregelix internal use only
-	 * 
-	 * @param writers
-	 */
-	public void setOutputTupleBuilders(List<ArrayTupleBuilder> tbs) {
-		delegate.setOutputTupleBuilders(tbs);
-	}
+    /**
+     * Pregelix internal use only
+     * 
+     * @param writers
+     */
+    public void setOutputAppenders(List<FrameTupleAppender> appenders) {
+        delegate.setOutputAppenders(appenders);
+    }
 
-	/**
-	 * Pregelix internal use only
-	 * 
-	 * @param writers
-	 */
-	public void finishCompute() throws IOException {
-		delegate.finishCompute();
-	}
+    /**
+     * Pregelix internal use only
+     * 
+     * @param writers
+     */
+    public void setOutputTupleBuilders(List<ArrayTupleBuilder> tbs) {
+        delegate.setOutputTupleBuilders(tbs);
+    }
 
-	/**
-	 * Pregelix internal use only
-	 */
-	public boolean hasUpdate() {
-		return this.updated;
-	}
+    /**
+     * Pregelix internal use only
+     * 
+     * @param writers
+     */
+    public void finishCompute() throws IOException {
+        delegate.finishCompute();
+    }
 
-	/**
-	 * Pregelix internal use only
-	 */
-	public boolean hasMessage() {
-		return this.hasMessage;
-	}
+    /**
+     * Pregelix internal use only
+     */
+    public boolean hasUpdate() {
+        return this.updated;
+    }
 
-	/**
-	 * Pregelix internal use only
-	 */
-	public boolean createdNewLiveVertex() {
-		return this.createdNewLiveVertex;
-	}
+    /**
+     * Pregelix internal use only
+     */
+    public boolean hasMessage() {
+        return this.hasMessage;
+    }
 
-	/**
-	 * sort the edges
-	 */
-	@SuppressWarnings("unchecked")
-	public void sortEdges() {
-		updated = true;
-		Collections.sort(destEdgeList);
-	}
+    /**
+     * Pregelix internal use only
+     */
+    public boolean createdNewLiveVertex() {
+        return this.createdNewLiveVertex;
+    }
 
-	/**
-	 * 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;
-	}
+    /**
+     * sort the edges
+     */
+    @SuppressWarnings("unchecked")
+    public void sortEdges() {
+        updated = true;
+        Collections.sort(destEdgeList);
+    }
 
-	/**
-	 * 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 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;
+    }
 
-	/**
-	 * Set the global superstep for all the vertices (internal use)
-	 * 
-	 * @param superstep
-	 *            New superstep
-	 */
-	public static final void setSuperstep(long superstep) {
-		Vertex.superstep = superstep;
-	}
+    /**
+     * 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;
+    }
 
-	/**
-	 * 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);
-	}
+    /**
+     * Set the global superstep for all the vertices (internal use)
+     * 
+     * @param superstep
+     *            New superstep
+     */
+    public static final void setSuperstep(long superstep) {
+        Vertex.superstep = superstep;
+    }
 
-	/**
-	 * 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 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);
+    }
 
-	/**
-	 * 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);
-	}
+    /**
+     * 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);
+    }
 
-	/**
-	 * Delete a vertex from id
-	 * 
-	 * @param vertexId
-	 *            the vertex id
-	 */
-	public final void deleteVertex(I vertexId) {
-		delegate.deleteVertex(vertexId);
-	}
+    /**
+     * 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);
+    }
 
-	public void activate() {
-		if (halt)
-			this.updated = true;
-		this.halt = false;
-	}
+    /**
+     * 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 576758b..7b247a8 100644
--- a/pregelix/pregelix-core/pom.xml
+++ b/pregelix/pregelix-core/pom.xml
@@ -18,8 +18,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>
@@ -72,7 +73,7 @@
 			<plugin>
 				<groupId>org.codehaus.mojo</groupId>
 				<artifactId>appassembler-maven-plugin</artifactId>
-				<version>1.3</version>
+                <version>1.3</version>
 				<executions>
 					<execution>
 						<configuration>
@@ -164,7 +165,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 3a4c41b..3c9281a 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
@@ -22,9 +22,6 @@
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.UUID;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -33,6 +30,7 @@
 
 import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
@@ -45,7 +43,6 @@
 import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoinSingleSort;
 import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoinSort;
 import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
-import edu.uci.ics.pregelix.core.util.Utilities;
 import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
 
 @SuppressWarnings("rawtypes")
@@ -54,9 +51,7 @@
     private JobGen jobGen;
     private boolean profiling;
 
-    private String applicationName;
     private IHyracksClientConnection hcc;
-
     private Class exampleClass;
 
     public Driver(Class exampleClass) {
@@ -71,7 +66,6 @@
     @Override
     public void runJob(PregelixJob job, Plan planChoice, String ipAddress, int port, boolean profiling)
             throws HyracksException {
-        applicationName = exampleClass.getSimpleName() + UUID.randomUUID();
         try {
             /** add hadoop configurations */
             URL hadoopCore = job.getClass().getClassLoader().getResource("core-site.xml");
@@ -121,13 +115,13 @@
             for (URL url : urls)
                 if (url.toString().endsWith(".jar"))
                     jars.add(new File(url.getPath()));
-            installApplication(jars);
+            DeploymentId deploymentId = installApplication(jars);
 
             start = System.currentTimeMillis();
             FileSystem dfs = FileSystem.get(job.getConfiguration());
             dfs.delete(FileOutputFormat.getOutputPath(job), true);
-            runCreate(jobGen);
-            runDataLoad(jobGen);
+            runCreate(deploymentId, jobGen);
+            runDataLoad(deploymentId, jobGen);
             end = System.currentTimeMillis();
             time = end - start;
             LOG.info("data loading finished " + time + "ms");
@@ -135,7 +129,7 @@
             boolean terminate = false;
             do {
                 start = System.currentTimeMillis();
-                runLoopBodyIteration(jobGen, i);
+                runLoopBodyIteration(deploymentId, jobGen, i);
                 end = System.currentTimeMillis();
                 time = end - start;
                 LOG.info("iteration " + i + " finished " + time + "ms");
@@ -145,12 +139,12 @@
             } while (!terminate);
 
             start = System.currentTimeMillis();
-            runHDFSWRite(jobGen);
-            runCleanup(jobGen);
-            destroyApplication(applicationName);
+            runHDFSWRite(deploymentId, jobGen);
+            runCleanup(deploymentId, jobGen);
             end = System.currentTimeMillis();
             time = end - start;
             LOG.info("result writing finished " + time + "ms");
+            hcc.unDeployBinary(deploymentId);
             LOG.info("job finished");
         } catch (Exception e) {
             try {
@@ -167,84 +161,75 @@
         }
     }
 
-    private void runCreate(JobGen jobGen) throws Exception {
+    private void runCreate(DeploymentId deploymentId, JobGen jobGen) throws Exception {
         try {
             JobSpecification treeCreateSpec = jobGen.generateCreatingJob();
-            execute(treeCreateSpec);
+            execute(deploymentId, treeCreateSpec);
         } catch (Exception e) {
             throw e;
         }
     }
 
-    private void runDataLoad(JobGen jobGen) throws Exception {
+    private void runDataLoad(DeploymentId deploymentId, JobGen jobGen) throws Exception {
         try {
             JobSpecification bulkLoadJobSpec = jobGen.generateLoadingJob();
-            execute(bulkLoadJobSpec);
+            execute(deploymentId, bulkLoadJobSpec);
         } catch (Exception e) {
             throw e;
         }
     }
 
-    private void runLoopBodyIteration(JobGen jobGen, int iteration) throws Exception {
+    private void runLoopBodyIteration(DeploymentId deploymentId, JobGen jobGen, int iteration) throws Exception {
         try {
             JobSpecification loopBody = jobGen.generateJob(iteration);
-            execute(loopBody);
+            execute(deploymentId, loopBody);
         } catch (Exception e) {
             throw e;
         }
     }
 
-    private void runHDFSWRite(JobGen jobGen) throws Exception {
+    private void runHDFSWRite(DeploymentId deploymentId, JobGen jobGen) throws Exception {
         try {
             JobSpecification scanSortPrintJobSpec = jobGen.scanIndexWriteGraph();
-            execute(scanSortPrintJobSpec);
+            execute(deploymentId, scanSortPrintJobSpec);
         } catch (Exception e) {
             throw e;
         }
     }
 
-    private void runCleanup(JobGen jobGen) throws Exception {
+    private void runCleanup(DeploymentId deploymentId, JobGen jobGen) throws Exception {
         try {
             JobSpecification[] cleanups = jobGen.generateCleanup();
-            runJobArray(cleanups);
+            runJobArray(deploymentId, cleanups);
         } catch (Exception e) {
             throw e;
         }
     }
 
-    private void runJobArray(JobSpecification[] jobs) throws Exception {
+    private void runJobArray(DeploymentId deploymentId, JobSpecification[] jobs) throws Exception {
         for (JobSpecification job : jobs) {
-            execute(job);
+            execute(deploymentId, job);
         }
     }
 
-    private void execute(JobSpecification job) throws Exception {
+    private void execute(DeploymentId deploymentId, JobSpecification job) throws Exception {
         job.setUseConnectorPolicyForScheduling(false);
-        JobId jobId = hcc.startJob(applicationName, job,
+        JobId jobId = hcc.startJob(deploymentId, job,
                 profiling ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
         hcc.waitForCompletion(jobId);
     }
 
-    public void installApplication(List<File> jars) throws Exception {
-        Set<String> allJars = new TreeSet<String>();
+    public DeploymentId installApplication(List<File> jars) throws Exception {
+        List<String> allJars = new ArrayList<String>();
         for (File jar : jars) {
             allJars.add(jar.getAbsolutePath());
         }
         long start = System.currentTimeMillis();
-        File appZip = Utilities.getHyracksArchive(applicationName, allJars);
+        DeploymentId deploymentId = hcc.deployBinary(allJars);
         long end = System.currentTimeMillis();
-        LOG.info("jar packing finished " + (end - start) + "ms");
-
-        start = System.currentTimeMillis();
-        hcc.createApplication(applicationName, appZip);
-        end = System.currentTimeMillis();
         LOG.info("jar deployment finished " + (end - start) + "ms");
+        return deploymentId;
     }
-
-    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/hadoop/config/ConfigurationFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java
index f3089ba..d225eb4 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java
@@ -17,6 +17,7 @@
 
 import org.apache.hadoop.conf.Configuration;
 
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.pregelix.api.util.SerDeUtils;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
@@ -34,11 +35,11 @@
     }
 
     @Override
-    public Configuration createConfiguration() throws HyracksDataException {
+    public Configuration createConfiguration(IHyracksTaskContext ctx) throws HyracksDataException {
         try {
             Configuration conf = new Configuration();
+            conf.setClassLoader(ctx.getJobletContext().getClassLoader());
             SerDeUtils.deserialize(conf, data);
-            conf.setClassLoader(this.getClass().getClassLoader());
             return conf;
         } catch (Exception e) {
             throw new HyracksDataException(e);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
index 0b1be61..5816c0c 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
@@ -73,7 +73,6 @@
 import edu.uci.ics.pregelix.core.jobgen.provider.NormalizedKeyComputerFactoryProvider;
 import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
-import edu.uci.ics.pregelix.core.util.DatatypeHelper;
 import edu.uci.ics.pregelix.dataflow.HDFSFileWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.VertexFileScanOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.VertexWriteOperatorDescriptor;
@@ -84,6 +83,7 @@
 import edu.uci.ics.pregelix.runtime.bootstrap.TreeIndexRegistryProvider;
 import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
+import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
 
 public abstract class JobGen implements IJobGen {
     private static final Logger LOGGER = Logger.getLogger(JobGen.class.getName());
@@ -238,7 +238,7 @@
          * connect operator descriptors
          */
         ITuplePartitionComputerFactory hashPartitionComputerFactory = new VertexIdPartitionComputerFactory(
-                DatatypeHelper.createSerializerDeserializer(vertexIdClass));
+                new WritableSerializerDeserializerFactory(vertexIdClass));
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0, sorter, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, btreeBulkLoad, 0);
         return spec;
@@ -314,7 +314,7 @@
          * connect operator descriptors
          */
         ITuplePartitionComputerFactory hashPartitionComputerFactory = new VertexIdPartitionComputerFactory(
-                DatatypeHelper.createSerializerDeserializer(vertexIdClass));
+                new WritableSerializerDeserializerFactory(vertexIdClass));
         spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, sorter, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
                 comparatorFactories), sorter, 0, writer, 0);
@@ -381,7 +381,7 @@
         int[] sortFields = new int[1];
         sortFields[0] = 0;
         ITuplePartitionComputerFactory hashPartitionComputerFactory = new VertexIdPartitionComputerFactory(
-                DatatypeHelper.createSerializerDeserializer(vertexIdClass));
+                new WritableSerializerDeserializerFactory(vertexIdClass));
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, scanner, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
                 comparatorFactories), scanner, 0, writer, 0);
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 9de4c04..6d8fc34 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
@@ -73,6 +73,7 @@
 import edu.uci.ics.pregelix.runtime.touchpoint.PreSuperStepRuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
+import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
 
 public class JobGenInnerJoin extends JobGen {
 
@@ -245,7 +246,7 @@
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                rdUnnestedMessage.getFields()[0]);
+                new WritableSerializerDeserializerFactory(vertexIdClass));
         ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
 
         /** connect all operators **/
@@ -470,7 +471,7 @@
 
         ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                rdUnnestedMessage.getFields()[0]);
+                new WritableSerializerDeserializerFactory(vertexIdClass));
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, materializeRead, 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 91c15b2..5fb0c14 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
@@ -73,6 +73,7 @@
 import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNullWriterFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
+import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
 
 public class JobGenOuterJoin extends JobGen {
 
@@ -221,14 +222,13 @@
 
         /** construct empty sink operator */
         EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
-        ClusterConfig.setLocationConstraint(spec, emptySink3);
 
         /** construct empty sink operator */
         EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                rdUnnestedMessage.getFields()[0]);
+                new WritableSerializerDeserializerFactory(vertexIdClass));
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
@@ -432,7 +432,7 @@
 
         ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                rdUnnestedMessage.getFields()[0]);
+                new WritableSerializerDeserializerFactory(vertexIdClass));
 
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 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 ee1fd0f..a3b4ef9 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
@@ -72,6 +72,7 @@
 import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNullWriterFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
+import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
 
 public class JobGenOuterJoinSingleSort extends JobGen {
 
@@ -221,7 +222,7 @@
 
         ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                rdUnnestedMessage.getFields()[0]);
+                new WritableSerializerDeserializerFactory(vertexIdClass));
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
@@ -408,7 +409,7 @@
 
         ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                rdUnnestedMessage.getFields()[0]);
+                new WritableSerializerDeserializerFactory(vertexIdClass));
 
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 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 628e9ce..0b30dbb 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
@@ -72,6 +72,7 @@
 import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNullWriterFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
+import edu.uci.ics.pregelix.runtime.touchpoint.WritableSerializerDeserializerFactory;
 
 public class JobGenOuterJoinSort extends JobGen {
 
@@ -234,7 +235,7 @@
         ClusterConfig.setLocationConstraint(spec, emptySink4);
 
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                rdUnnestedMessage.getFields()[0]);
+                new WritableSerializerDeserializerFactory(vertexIdClass));
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
@@ -441,7 +442,7 @@
 
         ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
         ITuplePartitionComputerFactory partionFactory = new VertexIdPartitionComputerFactory(
-                rdUnnestedMessage.getFields()[0]);
+                new WritableSerializerDeserializerFactory(vertexIdClass));
 
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableRecordDescriptorFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableRecordDescriptorFactory.java
index d1d927d..145169e 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableRecordDescriptorFactory.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableRecordDescriptorFactory.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.pregelix.core.runtime.touchpoint;
 
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
@@ -29,9 +30,9 @@
     }
 
     @Override
-    public RecordDescriptor createRecordDescriptor() throws HyracksDataException {
+    public RecordDescriptor createRecordDescriptor(IHyracksTaskContext ctx) throws HyracksDataException {
         try {
-            return DataflowUtils.getRecordDescriptorFromWritableClasses(fieldClasses);
+            return DataflowUtils.getRecordDescriptorFromWritableClasses(ctx, fieldClasses);
         } catch (HyracksException e) {
             throw new HyracksDataException(e);
         }
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DataflowUtils.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DataflowUtils.java
index bcf3ffc..e9132c4 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DataflowUtils.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DataflowUtils.java
@@ -17,6 +17,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
@@ -27,6 +28,7 @@
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.runtime.simpleagg.AccumulatingAggregatorFactory;
 import edu.uci.ics.pregelix.runtime.simpleagg.AggregationFunctionFactory;
+import edu.uci.ics.pregelix.runtime.touchpoint.DatatypeHelper;
 
 public class DataflowUtils {
 
@@ -40,9 +42,10 @@
             throws HyracksException {
         RecordDescriptor recordDescriptor = null;
         try {
+            ClassLoader loader = DataflowUtils.class.getClassLoader();
             recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
-                    (Class<? extends Writable>) Class.forName(className1),
-                    (Class<? extends Writable>) Class.forName(className2));
+                    (Class<? extends Writable>) loader.loadClass(className1),
+                    (Class<? extends Writable>) loader.loadClass(className2));
         } catch (ClassNotFoundException cnfe) {
             throw new HyracksException(cnfe);
         }
@@ -53,11 +56,12 @@
     public static RecordDescriptor getRecordDescriptorFromWritableClasses(String... classNames) throws HyracksException {
         RecordDescriptor recordDescriptor = null;
         ISerializerDeserializer[] serdes = new ISerializerDeserializer[classNames.length];
+        ClassLoader loader = DataflowUtils.class.getClassLoader();
         try {
             int i = 0;
             for (String className : classNames)
-                serdes[i++] = DatatypeHelper.createSerializerDeserializer((Class<? extends Writable>) Class
-                        .forName(className));
+                serdes[i++] = DatatypeHelper.createSerializerDeserializer((Class<? extends Writable>) loader
+                        .loadClass(className));
         } catch (ClassNotFoundException cnfe) {
             throw new HyracksException(cnfe);
         }
@@ -79,4 +83,35 @@
                 new IAggregateFunctionFactory[] { aggFuncFactory });
         return aggregatorFactory;
     }
+
+    @SuppressWarnings("unchecked")
+    public static RecordDescriptor getRecordDescriptorFromKeyValueClasses(IHyracksTaskContext ctx, String className1,
+            String className2) throws HyracksException {
+        RecordDescriptor recordDescriptor = null;
+        try {
+            recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor((Class<? extends Writable>) ctx
+                    .getJobletContext().loadClass(className1), (Class<? extends Writable>) ctx.getJobletContext()
+                    .loadClass(className2));
+        } catch (Exception cnfe) {
+            throw new HyracksException(cnfe);
+        }
+        return recordDescriptor;
+    }
+
+    @SuppressWarnings({ "unchecked", "rawtypes" })
+    public static RecordDescriptor getRecordDescriptorFromWritableClasses(IHyracksTaskContext ctx, String... classNames)
+            throws HyracksException {
+        RecordDescriptor recordDescriptor = null;
+        ISerializerDeserializer[] serdes = new ISerializerDeserializer[classNames.length];
+        try {
+            int i = 0;
+            for (String className : classNames)
+                serdes[i++] = DatatypeHelper.createSerializerDeserializer((Class<? extends Writable>) ctx
+                        .getJobletContext().loadClass(className));
+        } catch (Exception cnfe) {
+            throw new HyracksException(cnfe);
+        }
+        recordDescriptor = new RecordDescriptor(serdes);
+        return recordDescriptor;
+    }
 }
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 cd2a864..2173e10 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
@@ -27,6 +27,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 {
 
@@ -54,7 +55,6 @@
         ccConfig.defaultMaxJobAttempts = 0;
         ccConfig.jobHistorySize = 0;
         ccConfig.profileDumpPeriod = -1;
-        ccConfig.clusterTopologyDefinition = new File(topologyFilePath);
 
         // cluster controller
         cc = new ClusterControllerService(ccConfig);
@@ -68,6 +68,7 @@
         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();
 
@@ -78,6 +79,7 @@
         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();
 
@@ -86,14 +88,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();
@@ -102,7 +96,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/pregelixcc b/pregelix/pregelix-core/src/main/resources/scripts/pregelixcc
new file mode 100755
index 0000000..461d76a
--- /dev/null
+++ b/pregelix/pregelix-core/src/main/resources/scripts/pregelixcc
@@ -0,0 +1,117 @@
+#!/bin/sh
+# ----------------------------------------------------------------------------
+#  Copyright 2001-2006 The Apache Software Foundation.
+#
+#  Licensed under the Apache License, Version 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.
+# ----------------------------------------------------------------------------
+#
+#   Copyright (c) 2001-2006 The Apache Software Foundation.  All rights
+#   reserved.
+
+
+# resolve links - $0 may be a softlink
+PRG="$0"
+
+while [ -h "$PRG" ]; do
+  ls=`ls -ld "$PRG"`
+  link=`expr "$ls" : '.*-> \(.*\)$'`
+  if expr "$link" : '/.*' > /dev/null; then
+    PRG="$link"
+  else
+    PRG=`dirname "$PRG"`/"$link"
+  fi
+done
+
+PRGDIR=`dirname "$PRG"`
+BASEDIR=`cd "$PRGDIR/.." >/dev/null; pwd`
+
+
+
+# OS specific support.  $var _must_ be set to either true or false.
+cygwin=false;
+darwin=false;
+case "`uname`" in
+  CYGWIN*) cygwin=true ;;
+  Darwin*) darwin=true
+           if [ -z "$JAVA_VERSION" ] ; then
+             JAVA_VERSION="CurrentJDK"
+           else
+             echo "Using Java version: $JAVA_VERSION"
+           fi
+           if [ -z "$JAVA_HOME" ] ; then
+             JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Versions/${JAVA_VERSION}/Home
+           fi
+           ;;
+esac
+
+if [ -z "$JAVA_HOME" ] ; then
+  if [ -r /etc/gentoo-release ] ; then
+    JAVA_HOME=`java-config --jre-home`
+  fi
+fi
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched
+if $cygwin ; then
+  [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+  [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
+fi
+
+# If a specific java binary isn't specified search for the standard 'java' binary
+if [ -z "$JAVACMD" ] ; then
+  if [ -n "$JAVA_HOME"  ] ; then
+    if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
+      # IBM's JDK on AIX uses strange locations for the executables
+      JAVACMD="$JAVA_HOME/jre/sh/java"
+    else
+      JAVACMD="$JAVA_HOME/bin/java"
+    fi
+  else
+    JAVACMD=`which java`
+  fi
+fi
+
+if [ ! -x "$JAVACMD" ] ; then
+  echo "Error: JAVA_HOME is not defined correctly." 1>&2
+  echo "  We cannot execute $JAVACMD" 1>&2
+  exit 1
+fi
+
+if [ -z "$REPO" ]
+then
+  REPO="$BASEDIR"/lib
+fi
+
+CLASSPATH=$CLASSPATH_PREFIX:"$HADOOP_HOME"/conf:/etc/hadoop/conf:"$BASEDIR"/etc:$1
+
+for f in ${REPO}/*.jar; do
+  CLASSPATH=${CLASSPATH}:$f;
+done
+
+# For Cygwin, switch paths to Windows format before running java
+if $cygwin; then
+  [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --windows "$CLASSPATH"`
+  [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --path --windows "$JAVA_HOME"`
+  [ -n "$HOME" ] && HOME=`cygpath --path --windows "$HOME"`
+  [ -n "$BASEDIR" ] && BASEDIR=`cygpath --path --windows "$BASEDIR"`
+  [ -n "$REPO" ] && REPO=`cygpath --path --windows "$REPO"`
+fi
+
+exec "$JAVACMD" $JAVA_OPTS  \
+  -classpath "$CLASSPATH" \
+  -Dapp.name="pregelixcc" \
+  -Dapp.pid="$$" \
+  -Dapp.repo="$REPO" \
+  -Dapp.home="$BASEDIR" \
+  -Dbasedir="$BASEDIR" \
+  edu.uci.ics.hyracks.control.cc.CCDriver \
+  "$@"
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/pregelixnc b/pregelix/pregelix-core/src/main/resources/scripts/pregelixnc
new file mode 100755
index 0000000..4a6ca1f
--- /dev/null
+++ b/pregelix/pregelix-core/src/main/resources/scripts/pregelixnc
@@ -0,0 +1,118 @@
+#!/bin/sh
+# ----------------------------------------------------------------------------
+#  Copyright 2001-2006 The Apache Software Foundation.
+#
+#  Licensed under the Apache License, Version 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.
+# ----------------------------------------------------------------------------
+#
+#   Copyright (c) 2001-2006 The Apache Software Foundation.  All rights
+#   reserved.
+
+
+# resolve links - $0 may be a softlink
+PRG="$0"
+
+while [ -h "$PRG" ]; do
+  ls=`ls -ld "$PRG"`
+  link=`expr "$ls" : '.*-> \(.*\)$'`
+  if expr "$link" : '/.*' > /dev/null; then
+    PRG="$link"
+  else
+    PRG=`dirname "$PRG"`/"$link"
+  fi
+done
+
+PRGDIR=`dirname "$PRG"`
+BASEDIR=`cd "$PRGDIR/.." >/dev/null; pwd`
+
+
+
+# OS specific support.  $var _must_ be set to either true or false.
+cygwin=false;
+darwin=false;
+case "`uname`" in
+  CYGWIN*) cygwin=true ;;
+  Darwin*) darwin=true
+           if [ -z "$JAVA_VERSION" ] ; then
+             JAVA_VERSION="CurrentJDK"
+           else
+             echo "Using Java version: $JAVA_VERSION"
+           fi
+           if [ -z "$JAVA_HOME" ] ; then
+             JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Versions/${JAVA_VERSION}/Home
+           fi
+           ;;
+esac
+
+if [ -z "$JAVA_HOME" ] ; then
+  if [ -r /etc/gentoo-release ] ; then
+    JAVA_HOME=`java-config --jre-home`
+  fi
+fi
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched
+if $cygwin ; then
+  [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+  [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
+fi
+
+# If a specific java binary isn't specified search for the standard 'java' binary
+if [ -z "$JAVACMD" ] ; then
+  if [ -n "$JAVA_HOME"  ] ; then
+    if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
+      # IBM's JDK on AIX uses strange locations for the executables
+      JAVACMD="$JAVA_HOME/jre/sh/java"
+    else
+      JAVACMD="$JAVA_HOME/bin/java"
+    fi
+  else
+    JAVACMD=`which java`
+  fi
+fi
+
+if [ ! -x "$JAVACMD" ] ; then
+  echo "Error: JAVA_HOME is not defined correctly." 1>&2
+  echo "  We cannot execute $JAVACMD" 1>&2
+  exit 1
+fi
+
+if [ -z "$REPO" ]
+then
+  REPO="$BASEDIR"/lib
+fi
+
+CLASSPATH=$CLASSPATH_PREFIX:"$HADOOP_HOME"/conf:/etc/hadoop/conf:"$BASEDIR"/etc:$1
+
+for f in ${REPO}/*.jar; do
+  CLASSPATH=${CLASSPATH}:$f;
+done
+
+
+# For Cygwin, switch paths to Windows format before running java
+if $cygwin; then
+  [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --windows "$CLASSPATH"`
+  [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --path --windows "$JAVA_HOME"`
+  [ -n "$HOME" ] && HOME=`cygpath --path --windows "$HOME"`2
+  [ -n "$BASEDIR" ] && BASEDIR=`cygpath --path --windows "$BASEDIR"`
+  [ -n "$REPO" ] && REPO=`cygpath --path --windows "$REPO"`
+fi
+
+exec "$JAVACMD" $JAVA_OPTS  \
+  -classpath "$CLASSPATH" \
+  -Dapp.name="pregelixnc" \
+  -Dapp.pid="$$" \
+  -Dapp.repo="$REPO" \
+  -Dapp.home="$BASEDIR" \
+  -Dbasedir="$BASEDIR" \
+  edu.uci.ics.hyracks.control.nc.NCDriver \
+  -app-nc-main-class edu.uci.ics.pregelix.runtime.bootstrap.NCApplicationEntryPoint "$@"
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh b/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
index 133b604..74f0987 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
@@ -20,12 +20,15 @@
 export JAVA_HOME=$JAVA_HOME
 export JAVA_OPTS=$CCJAVA_OPTS
 
+PREGELIX_HOME=`pwd`
 
-chmod -R 755 $HYRACKS_HOME
+#Enter the temp dir
+cd $CCTMP_DIR
+
 if [ -f "conf/topology.xml"  ]; then
 #Launch hyracks cc script with topology
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
+${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
 else
 #Launch hyracks cc script without toplogy
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
+${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
 fi
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh b/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh
index b059aad..3ab1b25 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh
@@ -39,11 +39,10 @@
 #Set JAVA_OPTS
 export JAVA_OPTS=$NCJAVA_OPTS
 
-cd $HYRACKS_HOME
-HYRACKS_HOME=`pwd`
+PREGELIX_HOME=`pwd`
 
 #Enter the temp dir
 cd $NCTMP_DIR
 
 #Launch hyracks nc
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -result-ip-address $IPADDR  -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
+${PREGELIX_HOME}/bin/pregelixnc -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/stopcc.sh b/pregelix/pregelix-core/src/main/resources/scripts/stopcc.sh
index c2f525a..84f369f 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/stopcc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/stopcc.sh
@@ -2,7 +2,18 @@
 . conf/cluster.properties
 
 #Kill process
-PID=`ps -ef|grep ${USER}|grep java|grep hyracks|awk '{print $2}'`
+#Kill process
+PID=`ps -ef|grep ${USER}|grep java|grep 'Dapp.name=pregelixcc'|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=pregelixcc'|awk '{print $2}'`
+fi
+
 echo $PID
 kill -9 $PID
 
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh b/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh
index 35c4794..7a5352d 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh
@@ -2,7 +2,11 @@
 . conf/cluster.properties
 
 #Kill process
-PID=`ps -ef|grep ${USER}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
+PID=`ps -ef|grep ${USER}|grep java|grep 'Dapp.name=pregelixnc'|awk '{print $2}'`
+
+if [ "$PID" == "" ]; then
+  PID=`ps -ef|grep ${USER}|grep java|grep 'hyracks'|awk '{print $2}'`
+fi
 
 if [ "$PID" == "" ]; then
   PID=`ps -ef|grep ${USER}|grep java|grep 'hyracks'|awk '{print $2}'`
@@ -10,7 +14,7 @@
 
 if [ "$PID" == "" ]; then
   USERID=`id | sed 's/^uid=//;s/(.*$//'`
-  PID=`ps -ef|grep ${USERID}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
+  PID=`ps -ef|grep ${USERID}|grep java|grep 'Dapp.name=pregelixnc'|awk '{print $2}'`
 fi
 
 echo $PID
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 f7cadf6..572bff9 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
@@ -102,8 +102,7 @@
         ClusterConfig.setStorePath(PATH_TO_CLUSTER_STORE);
         ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
         cleanupStores();
-        PregelixHyracksIntegrationUtil.init("src/test/resources/topology.xml");
-        PregelixHyracksIntegrationUtil.createApp(HYRACKS_APP_NAME);
+        PregelixHyracksIntegrationUtil.init();
 
         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 eeaa6c9..6222a04 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-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IAggregateFunctionFactory.java b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IAggregateFunctionFactory.java
index 4be0bed..dc75b07 100644
--- a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IAggregateFunctionFactory.java
+++ b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IAggregateFunctionFactory.java
@@ -16,9 +16,11 @@
 
 import java.io.Serializable;
 
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
 
 public interface IAggregateFunctionFactory extends Serializable {
-    public IAggregateFunction createAggregateFunction(IDataOutputProvider provider) throws HyracksException;
+	public IAggregateFunction createAggregateFunction(IHyracksTaskContext ctx,
+			IDataOutputProvider provider) throws HyracksException;
 }
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IRecordDescriptorFactory.java b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IRecordDescriptorFactory.java
index e7de650..7454f2d 100644
--- a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IRecordDescriptorFactory.java
+++ b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IRecordDescriptorFactory.java
@@ -16,11 +16,12 @@
 
 import java.io.Serializable;
 
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface IRecordDescriptorFactory extends Serializable {
 
-    public RecordDescriptor createRecordDescriptor() throws HyracksDataException;
+    public RecordDescriptor createRecordDescriptor(IHyracksTaskContext ctx) throws HyracksDataException;
 
 }
diff --git a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializerDeserializerFactory.java b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializerDeserializerFactory.java
new file mode 100644
index 0000000..16b067a
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializerDeserializerFactory.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.pregelix.dataflow.std.base;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+
+public interface ISerializerDeserializerFactory<T> extends Serializable {
+
+	public ISerializerDeserializer<T> getSerializerDeserializer();
+
+}
diff --git a/pregelix/pregelix-dataflow-std/pom.xml b/pregelix/pregelix-dataflow-std/pom.xml
index 889c876..aa77ad3 100644
--- a/pregelix/pregelix-dataflow-std/pom.xml
+++ b/pregelix/pregelix-dataflow-std/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/src/main/java/edu/uci/ics/pregelix/dataflow/std/FunctionCallOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FunctionCallOperatorDescriptor.java
index 4cbd6c4..24d28b0 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FunctionCallOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/FunctionCallOperatorDescriptor.java
@@ -66,7 +66,7 @@
             @Override
             public void open() throws HyracksDataException {
                 rd0 = inputRdFactory == null ? recordDescProvider.getInputRecordDescriptor(getActivityId(), 0)
-                        : inputRdFactory.createRecordDescriptor();
+                        : inputRdFactory.createRecordDescriptor(ctx);
                 frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), rd0);
                 ctxCL = Thread.currentThread().getContextClassLoader();
                 Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
index 99bca1a..f75dab2 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
@@ -56,7 +56,7 @@
      * @throws HyracksDataException
      */
     public void functionOpen() throws HyracksDataException {
-        inputRd = inputRdFactory.createRecordDescriptor();
+        inputRd = inputRdFactory.createRecordDescriptor(ctx);
         tupleDe = new TupleDeserializer(inputRd);
         ctxCL = Thread.currentThread().getContextClassLoader();
         Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
diff --git a/pregelix/pregelix-dataflow/pom.xml b/pregelix/pregelix-dataflow/pom.xml
index ffc000b..37abe57 100644
--- a/pregelix/pregelix-dataflow/pom.xml
+++ b/pregelix/pregelix-dataflow/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/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java
index eda7754..71592ee 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java
@@ -56,11 +56,11 @@
     public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         return new AbstractUnaryInputSinkOperatorNodePushable() {
-            private Configuration conf = confFactory.createConfiguration();
+            private Configuration conf = confFactory.createConfiguration(ctx);
             @SuppressWarnings("rawtypes")
             private GlobalAggregator aggregator = BspUtils.createGlobalAggregator(conf);
             private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(),
-                    inputRdFactory.createRecordDescriptor());
+                    inputRdFactory.createRecordDescriptor(ctx));
             private ByteBufferInputStream inputStream = new ByteBufferInputStream();
             private DataInput input = new DataInputStream(inputStream);
             private Writable partialAggregateValue = BspUtils.createFinalAggregateValue(conf);
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
index 2402748..1d9c778 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
@@ -75,14 +75,15 @@
             @Override
             public void open() throws HyracksDataException {
                 rd0 = inputRdFactory == null ? recordDescProvider.getInputRecordDescriptor(getActivityId(), 0)
-                        : inputRdFactory.createRecordDescriptor();
+                        : inputRdFactory.createRecordDescriptor(ctx);
                 frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), rd0);
                 ctxCL = Thread.currentThread().getContextClassLoader();
                 Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
-                conf = confFactory.createConfiguration();
+                conf = confFactory.createConfiguration(ctx);
 
                 VertexOutputFormat outputFormat = BspUtils.createVertexOutputFormat(conf);
                 context = ctxFactory.createContext(conf, partition);
+                context.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
                 try {
                     vertexWriter = outputFormat.createVertexWriter(context);
                 } catch (InterruptedException e) {
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
index b1bb555..9a87f02 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
@@ -66,14 +66,7 @@
                     } catch (Exception e) {
                         writer.fail();
                         throw new HyracksDataException(e);
-                    } finally {
-                        /**
-                         * remove last iteration's state
-                         */
-                        IterationUtils.removeIterationState(ctx, partition);
-                        writer.close();
                     }
-                    complete = true;
                 }
             }
 
@@ -84,7 +77,12 @@
 
             @Override
             public void close() throws HyracksDataException {
-
+                /**
+                 * remove last iteration's state
+                 */
+                IterationUtils.removeIterationState(ctx, partition);
+                writer.close();
+                complete = true;
             }
         };
     }
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/TerminationStateWriterOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/TerminationStateWriterOperatorDescriptor.java
index 88a0dda..f54d176 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/TerminationStateWriterOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/TerminationStateWriterOperatorDescriptor.java
@@ -45,10 +45,10 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         return new AbstractUnaryInputSinkOperatorNodePushable() {
-            private Configuration conf = confFactory.createConfiguration();
+            private Configuration conf = confFactory.createConfiguration(ctx);
             private boolean terminate = true;
 
             @Override
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java
index 0da7baf..cec5b55 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java
@@ -81,15 +81,13 @@
         final List<FileSplit> splits = splitsFactory.getSplits();
 
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
-            private ClassLoader ctxCL;
             private ContextFactory ctxFactory = new ContextFactory();
 
             @Override
             public void initialize() throws HyracksDataException {
                 ctxCL = Thread.currentThread().getContextClassLoader();
                 try {
-                    Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
-                    Configuration conf = confFactory.createConfiguration();
+                    Configuration conf = confFactory.createConfiguration(ctx);
                     writer.open();
                     for (int i = 0; i < scheduledLocations.length; i++) {
                         if (scheduledLocations[i].equals(ctx.getJobletContext().getApplicationContext().getNodeId())) {
@@ -135,10 +133,11 @@
                 VertexInputFormat vertexInputFormat = BspUtils.createVertexInputFormat(conf);
                 InputSplit split = splits.get(splitId);
                 TaskAttemptContext mapperContext = ctxFactory.createContext(conf, splitId);
+                mapperContext.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
 
                 VertexReader vertexReader = vertexInputFormat.createVertexReader(split, mapperContext);
                 vertexReader.initialize(split, mapperContext);
-                Vertex readerVertex = (Vertex) BspUtils.createVertex(conf);
+                Vertex readerVertex = (Vertex) BspUtils.createVertex(mapperContext.getConfiguration());
                 ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldSize);
                 DataOutput dos = tb.getDataOutput();
 
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexWriteOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexWriteOperatorDescriptor.java
index d7cbb3a..5da0239 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexWriteOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexWriteOperatorDescriptor.java
@@ -64,7 +64,7 @@
             @Override
             public void open() throws HyracksDataException {
                 rd0 = inputRdFactory == null ? recordDescProvider.getInputRecordDescriptor(getActivityId(), 0)
-                        : inputRdFactory.createRecordDescriptor();
+                        : inputRdFactory.createRecordDescriptor(ctx);
                 frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), rd0);
                 try {
                     outputWriter = new PrintWriter(new OutputStreamWriter(new FileOutputStream(splits[partition]
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/base/IConfigurationFactory.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/base/IConfigurationFactory.java
index b31f376..2e58196 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/base/IConfigurationFactory.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/base/IConfigurationFactory.java
@@ -18,10 +18,11 @@
 
 import org.apache.hadoop.conf.Configuration;
 
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface IConfigurationFactory extends Serializable {
 
-    public Configuration createConfiguration() throws HyracksDataException;
+    public Configuration createConfiguration(IHyracksTaskContext ctx) throws HyracksDataException;
 
 }
diff --git a/pregelix/pregelix-example/data/clique2/clique.txt b/pregelix/pregelix-example/data/clique2/clique.txt
new file mode 100755
index 0000000..68fbaea
--- /dev/null
+++ b/pregelix/pregelix-example/data/clique2/clique.txt
@@ -0,0 +1,6 @@
+1 2 3 4
+2 1 3
+3 1 2 4 5
+4 1 3
+5 3 6
+6 5
\ No newline at end of file
diff --git a/pregelix/pregelix-example/data/clique3/clique.txt b/pregelix/pregelix-example/data/clique3/clique.txt
new file mode 100755
index 0000000..ed0b7c1
--- /dev/null
+++ b/pregelix/pregelix-example/data/clique3/clique.txt
@@ -0,0 +1,20 @@
+0 1 19
+1 2 3 4 5 6 7 8 9
+2 1 3 4 5 6 7 8 9
+3 1 2 4 5 6 7 8 9
+4 1 2 3 5 6 7 8 9
+5 1 2 3 4 6 7 8 9
+6 1 2 3 4 5 7 8 9
+7 1 2 3 4 5 6 8 9
+8 1 2 3 4 5 6 7 9
+9 1 2 3 4 5 6 7 8 10
+10 9 11
+11 10 12 13 14 15 16 17 18 19
+12 11 13 14 15 16 17 18 19
+13 11 12 14 15 16 17 18 19
+14 11 12 13 15 16 17 18 19
+15 11 12 13 14 16 17 18 19
+16 11 12 13 14 15 17 18 19
+17 11 12 13 14 15 16 18 19
+18 11 12 13 14 15 16 17 19
+19 0 11 12 13 14 15 16 17 18
\ No newline at end of file
diff --git a/pregelix/pregelix-example/pom.xml b/pregelix/pregelix-example/pom.xml
index beefee2..0c4e277 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>
@@ -68,7 +69,7 @@
 				<version>2.7.2</version>
 				<configuration>
 					<forkMode>pertest</forkMode>
-					<argLine>-enableassertions -Xmx512m -XX:MaxPermSize=300m -Dfile.encoding=UTF-8
+					<argLine>-enableassertions -Xmx2047m -XX:MaxPermSize=300m -Dfile.encoding=UTF-8
 						-Djava.util.logging.config.file=src/test/resources/logging.properties</argLine>
 					<includes>
 						<include>**/*TestSuite.java</include>
@@ -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/inputformat/TextPageRankInputFormat.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextPageRankInputFormat.java
index a8a752e..8d0b776 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextPageRankInputFormat.java
@@ -39,7 +39,7 @@
     @Override
     public VertexReader<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> createVertexReader(
             InputSplit split, TaskAttemptContext context) throws IOException {
-        return new TextPageRankGraphReader(textInputFormat.createRecordReader(split, context));
+        return new TextPageRankGraphReader(textInputFormat.createRecordReader(split, context), context);
     }
 }
 
@@ -52,7 +52,7 @@
     private List<VLongWritable> pool = new ArrayList<VLongWritable>();
     private int used = 0;
 
-    public TextPageRankGraphReader(RecordReader<LongWritable, Text> lineRecordReader) {
+    public TextPageRankGraphReader(RecordReader<LongWritable, Text> lineRecordReader, TaskAttemptContext context) {
         super(lineRecordReader);
     }
 
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/CliquesWritable.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/CliquesWritable.java
index 0e22ea1..d2c5e6f 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/CliquesWritable.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/CliquesWritable.java
@@ -32,6 +32,7 @@
 
     private List<VLongWritable> cliques = new ArrayList<VLongWritable>();
     private int sizeOfClique = 0;
+    private VLongWritable srcId = new VLongWritable(0);
 
     public CliquesWritable(List<VLongWritable> cliques, int sizeOfClique) {
         this.cliques = cliques;
@@ -43,6 +44,16 @@
     }
 
     /**
+     * Set the srcId
+     * 
+     * @param srcId
+     *            the source vertex Id
+     */
+    public void setSrcId(VLongWritable srcId) {
+        this.srcId = srcId;
+    }
+
+    /**
      * Set the size of cliques.
      * 
      * @param sizeOfClique
@@ -103,6 +114,11 @@
                 cliques.add(vid);
             }
         }
+
+        if (srcId == null) {
+            srcId = new VLongWritable();
+        }
+        srcId.readFields(input);
     }
 
     @Override
@@ -117,6 +133,8 @@
         for (int i = 0; i < cliques.size(); i++) {
             cliques.get(i).write(output);
         }
+
+        srcId.write(output);
     }
 
     @Override
@@ -126,11 +144,14 @@
         StringBuffer sb = new StringBuffer();
         int numCliques = cliques.size() / sizeOfClique;
         for (int i = 0; i < numCliques; i++) {
+            sb.append(srcId);
+            sb.append(",");
+            int start = i * sizeOfClique;
             for (int j = 0; j < sizeOfClique - 1; j++) {
-                sb.append(cliques.get(j));
+                sb.append(cliques.get(start + j));
                 sb.append(",");
             }
-            sb.append(cliques.get(sizeOfClique - 1));
+            sb.append(cliques.get(start + sizeOfClique - 1));
             sb.append(";");
         }
         return sb.toString();
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java
index 266feb7..85a139e 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java
@@ -44,8 +44,8 @@
 
 /**
  * The maximal clique example -- find maximal cliques in an undirected graph.
- * The result cliques contains vertexes ordered by the vertex id ascendingly. The algorithm takes
- * advantage of that property to do effective pruning.
+ * The result cliques contains vertexes ordered by the vertex id ascendingly.
+ * The algorithm takes advantage of that property to do effective pruning.
  */
 public class MaximalCliqueVertex extends Vertex<VLongWritable, CliquesWritable, NullWritable, AdjacencyListWritable> {
 
@@ -55,7 +55,7 @@
     private int largestCliqueSizeSoFar = 0;
     private List<BitSet> currentMaximalCliques = new ArrayList<BitSet>();
     private CliquesWritable tmpValue = new CliquesWritable();
-    private List<VLongWritable> cliques = new ArrayList<VLongWritable>();
+    private List<VLongWritable> cliqueList = new ArrayList<VLongWritable>();
 
     /**
      * Update the current maximal cliques
@@ -68,7 +68,6 @@
         vertexList.clear();
         invertedMap.clear();
         currentMaximalCliques.clear();
-        cliques.clear();
         tmpValue.reset();
 
         // build the initial sub graph
@@ -76,8 +75,6 @@
             AdjacencyListWritable adj = values.next();
             map.put(adj.getSource(), adj);
         }
-        VLongWritable srcId = getVertexId();
-        map.put(srcId, new AdjacencyListWritable());
 
         // build the vertex list (vertex id in ascending order) and the inverted list of vertexes
         int i = 0;
@@ -86,12 +83,14 @@
             invertedMap.put(v, i++);
         }
 
-        //clean up adjacency list --- remove vertexes who are not neighbors of key
+        // clean up adjacency list --- remove vertexes who are not neighbors of
+        // key
         for (AdjacencyListWritable adj : map.values()) {
             adj.cleanNonMatch(vertexList);
         }
 
-        // get the h-index of the subgraph --- which is the maximum depth to explore
+        // get the h-index of the subgraph --- which is the maximum depth to
+        // explore
         int[] neighborCounts = new int[map.size()];
         i = 0;
         for (AdjacencyListWritable adj : map.values()) {
@@ -105,11 +104,13 @@
             }
             h++;
         }
-        if (h < largestCliqueSizeSoFar) {
+
+        // the clique size is upper-bounded by h+1
+        if (h + 1 < largestCliqueSizeSoFar) {
             return;
         }
 
-        //start depth-first search
+        // start depth-first search
         BitSet cliqueSoFar = new BitSet(h);
         for (VLongWritable v : vertexList) {
             cliqueSoFar.set(invertedMap.get(v));
@@ -117,16 +118,15 @@
             cliqueSoFar.clear();
         }
 
-        //output local maximal cliques
+        // output local maximal cliques
+        tmpValue.setSrcId(getVertexId());
         for (BitSet clique : currentMaximalCliques) {
-            int keyIndex = invertedMap.get(srcId);
-            clique.set(keyIndex);
             generateClique(clique);
-            tmpValue.addCliques(cliques);
+            tmpValue.addCliques(cliqueList);
             tmpValue.setCliqueSize(clique.cardinality());
         }
 
-        //update the vertex state
+        // update the vertex state
         setVertexValue(tmpValue);
     }
 
@@ -136,13 +136,15 @@
      * @param clique
      *            the bitmap representation of a clique
      */
-    private void generateClique(BitSet clique) {
+    private List<VLongWritable> generateClique(BitSet clique) {
+        cliqueList.clear();
         for (int j = 0; j < clique.length();) {
             j = clique.nextSetBit(j);
             VLongWritable v = vertexList.get(j);
-            cliques.add(v);
+            cliqueList.add(v);
             j++;
         }
+        return cliqueList;
     }
 
     /**
@@ -170,7 +172,7 @@
         while (neighbors.hasNext()) {
             VLongWritable neighbor = neighbors.next();
             if (!isTested(neighbor, cliqueSoFar) && isClique(neighbor, cliqueSoFar)) {
-                //snapshot the clique
+                // snapshot the clique
                 int cliqueLength = cliqueSoFar.length();
                 // expand the clique
                 cliqueSoFar.set(invertedMap.get(neighbor));
@@ -217,7 +219,8 @@
         int largestSetIndex = cliqueSoFar.length() - 1;
         if (index > largestSetIndex) {
             // we only return cliques with vertexes in the ascending order
-            // hence, the new vertex must be larger than the largesetSetIndex in the clique
+            // hence, the new vertex must be larger than the largesetSetIndex in
+            // the clique
             return false;
         } else {
             // otherwise, we think the vertex is "tested"
@@ -236,7 +239,8 @@
      */
     private boolean isClique(VLongWritable newVertex, BitSet cliqueSoFar) {
         AdjacencyListWritable adj = map.get(newVertex);
-        // check whether each existing vertex is in the neighbor set of newVertex
+        // check whether each existing vertex is in the neighbor set of
+        // newVertex
         for (int i = 0; i < cliqueSoFar.length();) {
             i = cliqueSoFar.nextSetBit(i);
             VLongWritable v = vertexList.get(i);
@@ -249,9 +253,8 @@
     }
 
     /**
-     * For superstep 1, send outgoing mesages.
-     * For superstep 2, calculate maximal cliques.
-     * otherwise, vote to halt.
+     * For superstep 1, send outgoing mesages. For superstep 2, calculate
+     * maximal cliques. otherwise, vote to halt.
      */
     @Override
     public void compute(Iterator<AdjacencyListWritable> msgIterator) {
@@ -300,9 +303,11 @@
     private void sendOutgoingMsgs(List<Edge<VLongWritable, NullWritable>> edges) {
         for (int i = 0; i < edges.size(); i++) {
             if (edges.get(i).getDestVertexId().get() < getVertexId().get()) {
-                // only add emit for the vertexes whose id is smaller than the vertex id 
+                // only add emit for the vertexes whose id is smaller than the
+                // vertex id
                 // to avoid the duplicate removal step,
-                // because all the resulting cliques will have vertexes in the ascending order.
+                // because all the resulting cliques will have vertexes in the
+                // ascending order.
                 AdjacencyListWritable msg = new AdjacencyListWritable();
                 msg.setSource(getVertexId());
                 for (int j = i + 1; j < edges.size(); j++) {
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 321b5b2..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
@@ -76,8 +76,7 @@
         ClusterConfig.setStorePath(PATH_TO_CLUSTER_STORE);
         ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
         cleanupStores();
-        PregelixHyracksIntegrationUtil.init("src/test/resources/topology.xml");
-        PregelixHyracksIntegrationUtil.createApp(HYRACKS_APP_NAME);
+        PregelixHyracksIntegrationUtil.init();
         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/jobgen/JobGenerator.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
index ca5a1c4..0a5b214 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
@@ -55,6 +55,8 @@
     private static String HDFS_OUTPUTPAH2 = "/resultcomplex";
 
     private static String HDFS_INPUTPATH3 = "/clique";
+    private static String HDFS_INPUTPATH4 = "/clique2";
+    private static String HDFS_INPUTPATH5 = "/clique3";
     private static String HDFS_OUTPUTPAH3 = "/resultclique";
 
     private static void generatePageRankJobReal(String jobName, String outputPath) throws IOException {
@@ -218,6 +220,30 @@
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
+    
+    private static void generateMaximalCliqueJob2(String jobName, String outputPath) throws IOException {
+        PregelixJob job = new PregelixJob(jobName);
+        job.setVertexClass(MaximalCliqueVertex.class);
+        job.setGlobalAggregatorClass(MaximalCliqueAggregator.class);
+        job.setDynamicVertexValueSize(true);
+        job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
+        job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
+        FileInputFormat.setInputPaths(job, HDFS_INPUTPATH4);
+        FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
+        job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+    }
+    
+    private static void generateMaximalCliqueJob3(String jobName, String outputPath) throws IOException {
+        PregelixJob job = new PregelixJob(jobName);
+        job.setVertexClass(MaximalCliqueVertex.class);
+        job.setGlobalAggregatorClass(MaximalCliqueAggregator.class);
+        job.setDynamicVertexValueSize(true);
+        job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
+        job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
+        FileInputFormat.setInputPaths(job, HDFS_INPUTPATH5);
+        FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
+        job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+    }
 
     private static void generateGraphMutationJob(String jobName, String outputPath) throws IOException {
         PregelixJob job = new PregelixJob(jobName);
@@ -261,6 +287,8 @@
 
     private static void genMaximalClique() throws IOException {
         generateMaximalCliqueJob("Maximal Clique", outputBase + "MaximalClique.xml");
+        generateMaximalCliqueJob2("Maximal Clique 2", outputBase + "MaximalClique2.xml");
+        generateMaximalCliqueJob3("Maximal Clique 3", outputBase + "MaximalClique3.xml");
     }
 
     private static void genGraphMutation() throws IOException {
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
index 5a556fa..00f6f54 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
@@ -25,20 +25,13 @@
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.junit.Test;
 
-import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
-import edu.uci.ics.pregelix.core.jobgen.JobGen;
-import edu.uci.ics.pregelix.core.jobgen.JobGenInnerJoin;
-import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoin;
-import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoinSingleSort;
-import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoinSort;
+import edu.uci.ics.pregelix.core.base.IDriver.Plan;
+import edu.uci.ics.pregelix.core.driver.Driver;
 import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
-import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
 import edu.uci.ics.pregelix.example.util.TestUtils;
 
 public class RunJobTestCase extends TestCase {
-    private static final String NC1 = "nc1";
-    private static final String HYRACKS_APP_NAME = "pregelix";
     private static String HDFS_INPUTPATH = "/webmap";
     private static String HDFS_OUTPUTPAH = "/result";
 
@@ -48,14 +41,21 @@
     private static String HDFS_INPUTPATH3 = "/clique";
     private static String HDFS_OUTPUTPAH3 = "/resultclique";
 
-    private final PregelixJob job;
-    private JobGen[] giraphJobGens;
-    private final String resultFileName;
-    private final String expectedFileName;
-    private final String jobFile;
+    private static String HDFS_INPUTPATH4 = "/clique2";
+    private static String HDFS_OUTPUTPAH4 = "/resultclique";
 
-    public RunJobTestCase(String hadoopConfPath, String jobName, String jobFile, String resultFile, String expectedFile)
-            throws Exception {
+    private static String HDFS_INPUTPATH5 = "/clique3";
+    private static String HDFS_OUTPUTPAH5 = "/resultclique";
+
+    private final PregelixJob job;
+    private final String resultFileDir;
+    private final String expectedFileDir;
+    private final String jobFile;
+    private final Driver driver = new Driver(this.getClass());
+    private final FileSystem dfs;
+
+    public RunJobTestCase(String hadoopConfPath, String jobName, String jobFile, String resultFile,
+            String expectedFile, FileSystem dfs) throws Exception {
         super("test");
         this.jobFile = jobFile;
         this.job = new PregelixJob("test");
@@ -68,21 +68,20 @@
         } else if (inputPaths[0].toString().endsWith(HDFS_INPUTPATH2)) {
             FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
             FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
-        } else {
+        } else if (inputPaths[0].toString().endsWith(HDFS_INPUTPATH3)) {
             FileInputFormat.setInputPaths(job, HDFS_INPUTPATH3);
             FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
+        } else if (inputPaths[0].toString().endsWith(HDFS_INPUTPATH4)) {
+            FileInputFormat.setInputPaths(job, HDFS_INPUTPATH4);
+            FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH4));
+        } else if (inputPaths[0].toString().endsWith(HDFS_INPUTPATH5)) {
+            FileInputFormat.setInputPaths(job, HDFS_INPUTPATH5);
+            FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH5));
         }
         job.setJobName(jobName);
-        this.resultFileName = resultFile;
-        this.expectedFileName = expectedFile;
-        giraphJobGens = new JobGen[4];
-        giraphJobGens[0] = new JobGenOuterJoin(job);
-        waitawhile();
-        giraphJobGens[1] = new JobGenInnerJoin(job);
-        waitawhile();
-        giraphJobGens[2] = new JobGenOuterJoinSort(job);
-        waitawhile();
-        giraphJobGens[3] = new JobGenOuterJoinSingleSort(job);
+        this.resultFileDir = resultFile;
+        this.expectedFileDir = expectedFile;
+        this.dfs = dfs;
     }
 
     private void waitawhile() throws InterruptedException {
@@ -94,89 +93,19 @@
     @Test
     public void test() throws Exception {
         setUp();
-        for (JobGen jobGen : giraphJobGens) {
-            FileSystem dfs = FileSystem.get(job.getConfiguration());
-            dfs.delete(new Path(HDFS_OUTPUTPAH), true);
-            runCreate(jobGen);
-            runDataLoad(jobGen);
-            int i = 1;
-            boolean terminate = false;
-            do {
-                runLoopBodyIteration(jobGen, i);
-                terminate = IterationUtils.readTerminationState(job.getConfiguration(), jobGen.getJobId());
-                i++;
-            } while (!terminate);
-            runIndexScan(jobGen);
-            runHDFSWRite(jobGen);
-            runCleanup(jobGen);
-            compareResults();
+        Plan[] plans = new Plan[] { Plan.OUTER_JOIN };
+        for (Plan plan : plans) {
+            driver.runJob(job, plan, PregelixHyracksIntegrationUtil.CC_HOST,
+                    PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT, false);
         }
+        compareResults();
         tearDown();
         waitawhile();
     }
 
-    private void runCreate(JobGen jobGen) throws Exception {
-        try {
-            JobSpecification treeCreateJobSpec = jobGen.generateCreatingJob();
-            PregelixHyracksIntegrationUtil.runJob(treeCreateJobSpec, HYRACKS_APP_NAME);
-        } catch (Exception e) {
-            throw e;
-        }
-    }
-
-    private void runDataLoad(JobGen jobGen) throws Exception {
-        try {
-            JobSpecification bulkLoadJobSpec = jobGen.generateLoadingJob();
-            PregelixHyracksIntegrationUtil.runJob(bulkLoadJobSpec, HYRACKS_APP_NAME);
-        } catch (Exception e) {
-            throw e;
-        }
-    }
-
-    private void runLoopBodyIteration(JobGen jobGen, int iteration) throws Exception {
-        try {
-            JobSpecification loopBody = jobGen.generateJob(iteration);
-            PregelixHyracksIntegrationUtil.runJob(loopBody, HYRACKS_APP_NAME);
-        } catch (Exception e) {
-            throw e;
-        }
-    }
-
-    private void runIndexScan(JobGen jobGen) throws Exception {
-        try {
-            JobSpecification scanSortPrintJobSpec = jobGen.scanIndexPrintGraph(NC1, resultFileName);
-            PregelixHyracksIntegrationUtil.runJob(scanSortPrintJobSpec, HYRACKS_APP_NAME);
-        } catch (Exception e) {
-            throw e;
-        }
-    }
-
-    private void runHDFSWRite(JobGen jobGen) throws Exception {
-        try {
-            JobSpecification scanSortPrintJobSpec = jobGen.scanIndexWriteGraph();
-            PregelixHyracksIntegrationUtil.runJob(scanSortPrintJobSpec, HYRACKS_APP_NAME);
-        } catch (Exception e) {
-            throw e;
-        }
-    }
-
-    private void runCleanup(JobGen jobGen) throws Exception {
-        try {
-            JobSpecification[] cleanups = jobGen.generateCleanup();
-            runJobArray(cleanups);
-        } catch (Exception e) {
-            throw e;
-        }
-    }
-
-    private void runJobArray(JobSpecification[] jobs) throws Exception {
-        for (JobSpecification job : jobs) {
-            PregelixHyracksIntegrationUtil.runJob(job, HYRACKS_APP_NAME);
-        }
-    }
-
     private void compareResults() throws Exception {
-        TestUtils.compareWithResult(new File(resultFileName), new File(expectedFileName));
+        dfs.copyToLocalFile(FileOutputFormat.getOutputPath(job), new Path(resultFileDir));
+        TestUtils.compareWithResultDir(new File(expectedFileDir), new File(resultFileDir));
     }
 
     public String toString() {
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 fa98ebd..a490049 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
@@ -41,176 +41,182 @@
 
 @SuppressWarnings("deprecation")
 public class RunJobTestSuite extends TestSuite {
-	private static final Logger LOGGER = Logger.getLogger(RunJobTestSuite.class
-			.getName());
+    private static final Logger LOGGER = Logger.getLogger(RunJobTestSuite.class.getName());
 
-	private static final String ACTUAL_RESULT_DIR = "actual";
-	private static final String EXPECTED_RESULT_DIR = "src/test/resources/expected";
-	private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
-	private static final String PATH_TO_CLUSTER_STORE = "src/test/resources/cluster/stores.properties";
-	private static final String PATH_TO_CLUSTER_PROPERTIES = "src/test/resources/cluster/cluster.properties";
-	private static final String PATH_TO_JOBS = "src/test/resources/jobs/";
-	private static final String PATH_TO_IGNORE = "src/test/resources/ignore.txt";
-	private static final String PATH_TO_ONLY = "src/test/resources/only.txt";
-	private static final String FILE_EXTENSION_OF_RESULTS = "result";
+    private static final String ACTUAL_RESULT_DIR = "actual";
+    private static final String EXPECTED_RESULT_DIR = "src/test/resources/expected";
+    private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
+    private static final String PATH_TO_CLUSTER_STORE = "src/test/resources/cluster/stores.properties";
+    private static final String PATH_TO_CLUSTER_PROPERTIES = "src/test/resources/cluster/cluster.properties";
+    private static final String PATH_TO_JOBS = "src/test/resources/jobs/";
+    private static final String PATH_TO_IGNORE = "src/test/resources/ignore.txt";
+    private static final String PATH_TO_ONLY = "src/test/resources/only.txt";
+    private static final String FILE_EXTENSION_OF_RESULTS = "result";
 
-	private static final String DATA_PATH = "data/webmap/webmap_link.txt";
-	private static final String HDFS_PATH = "/webmap/";
+    private static final String DATA_PATH = "data/webmap/webmap_link.txt";
+    private static final String HDFS_PATH = "/webmap/";
 
-	private static final String DATA_PATH2 = "data/webmapcomplex/webmap_link.txt";
-	private static final String HDFS_PATH2 = "/webmapcomplex/";
+    private static final String DATA_PATH2 = "data/webmapcomplex/webmap_link.txt";
+    private static final String HDFS_PATH2 = "/webmapcomplex/";
 
-	private static final String DATA_PATH3 = "data/clique/clique.txt";
-	private static final String HDFS_PATH3 = "/clique/";
+    private static final String DATA_PATH3 = "data/clique/clique.txt";
+    private static final String HDFS_PATH3 = "/clique/";
 
-	private static final String HYRACKS_APP_NAME = "pregelix";
-	private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR
-			+ File.separator + "conf.xml";
-	private MiniDFSCluster dfsCluster;
+    private static final String DATA_PATH4 = "data/clique2/clique.txt";
+    private static final String HDFS_PATH4 = "/clique2/";
 
-	private JobConf conf = new JobConf();
-	private int numberOfNC = 2;
+    private static final String DATA_PATH5 = "data/clique3/clique.txt";
+    private static final String HDFS_PATH5 = "/clique3/";
 
-	public void setUp() throws Exception {
-		ClusterConfig.setStorePath(PATH_TO_CLUSTER_STORE);
-		ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
-		cleanupStores();
-		PregelixHyracksIntegrationUtil.init("src/test/resources/topology.xml");
-		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));
-		startHDFS();
-	}
+    private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
+    private MiniDFSCluster dfsCluster;
 
-	private void cleanupStores() throws IOException {
-		FileUtils.forceMkdir(new File("teststore"));
-		FileUtils.forceMkdir(new File("build"));
-		FileUtils.cleanDirectory(new File("teststore"));
-		FileUtils.cleanDirectory(new File("build"));
-	}
+    private JobConf conf = new JobConf();
+    private int numberOfNC = 2;
 
-	private void startHDFS() throws IOException {
-		conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
-		conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
-		conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
-		FileSystem lfs = FileSystem.getLocal(new Configuration());
-		lfs.delete(new Path("build"), true);
-		System.setProperty("hadoop.log.dir", "logs");
-		dfsCluster = new MiniDFSCluster(conf, numberOfNC, true, null);
-		FileSystem dfs = FileSystem.get(conf);
-		Path src = new Path(DATA_PATH);
-		Path dest = new Path(HDFS_PATH);
-		dfs.mkdirs(dest);
-		dfs.copyFromLocalFile(src, dest);
+    public void setUp() throws Exception {
+        ClusterConfig.setStorePath(PATH_TO_CLUSTER_STORE);
+        ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
+        cleanupStores();
+        PregelixHyracksIntegrationUtil.init();
+        LOGGER.info("Hyracks mini-cluster started");
+        FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
+        FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
+        startHDFS();
+    }
 
-		src = new Path(DATA_PATH2);
-		dest = new Path(HDFS_PATH2);
-		dfs.mkdirs(dest);
-		dfs.copyFromLocalFile(src, dest);
+    private void cleanupStores() throws IOException {
+        FileUtils.forceMkdir(new File("teststore"));
+        FileUtils.forceMkdir(new File("build"));
+        FileUtils.cleanDirectory(new File("teststore"));
+        FileUtils.cleanDirectory(new File("build"));
+    }
 
-		src = new Path(DATA_PATH3);
-		dest = new Path(HDFS_PATH3);
-		dfs.mkdirs(dest);
-		dfs.copyFromLocalFile(src, dest);
+    private void startHDFS() throws IOException {
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
+        FileSystem lfs = FileSystem.getLocal(new Configuration());
+        lfs.delete(new Path("build"), true);
+        System.setProperty("hadoop.log.dir", "logs");
+        dfsCluster = new MiniDFSCluster(conf, numberOfNC, true, null);
+        FileSystem dfs = FileSystem.get(conf);
+        Path src = new Path(DATA_PATH);
+        Path dest = new Path(HDFS_PATH);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
 
-		DataOutputStream confOutput = new DataOutputStream(
-				new FileOutputStream(new File(HADOOP_CONF_PATH)));
-		conf.writeXml(confOutput);
-		confOutput.flush();
-		confOutput.close();
-	}
+        src = new Path(DATA_PATH2);
+        dest = new Path(HDFS_PATH2);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
 
-	/**
-	 * cleanup hdfs cluster
-	 */
-	private void cleanupHDFS() throws Exception {
-		dfsCluster.shutdown();
-	}
+        src = new Path(DATA_PATH3);
+        dest = new Path(HDFS_PATH3);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
 
-	public void tearDown() throws Exception {
-		PregelixHyracksIntegrationUtil.destroyApp(HYRACKS_APP_NAME);
-		PregelixHyracksIntegrationUtil.deinit();
-		LOGGER.info("Hyracks mini-cluster shut down");
-		cleanupHDFS();
-	}
+        src = new Path(DATA_PATH4);
+        dest = new Path(HDFS_PATH4);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
 
-	public static Test suite() throws Exception {
-		List<String> ignores = getFileList(PATH_TO_IGNORE);
-		List<String> onlys = getFileList(PATH_TO_ONLY);
-		File testData = new File(PATH_TO_JOBS);
-		File[] queries = testData.listFiles();
-		RunJobTestSuite testSuite = new RunJobTestSuite();
-		testSuite.setUp();
-		boolean onlyEnabled = false;
+        src = new Path(DATA_PATH5);
+        dest = new Path(HDFS_PATH5);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
 
-		if (onlys.size() > 0) {
-			onlyEnabled = true;
-		}
-		for (File qFile : queries) {
-			if (isInList(ignores, qFile.getName()))
-				continue;
+        DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
+        conf.writeXml(confOutput);
+        confOutput.flush();
+        confOutput.close();
+    }
 
-			if (qFile.isFile()) {
-				if (onlyEnabled && !isInList(onlys, qFile.getName())) {
-					continue;
-				} else {
-					String resultFileName = ACTUAL_RESULT_DIR + File.separator
-							+ jobExtToResExt(qFile.getName());
-					String expectedFileName = EXPECTED_RESULT_DIR
-							+ File.separator + jobExtToResExt(qFile.getName());
-					testSuite.addTest(new RunJobTestCase(HADOOP_CONF_PATH,
-							qFile.getName(),
-							qFile.getAbsolutePath().toString(), resultFileName,
-							expectedFileName));
-				}
-			}
-		}
-		return testSuite;
-	}
+    /**
+     * cleanup hdfs cluster
+     */
+    private void cleanupHDFS() throws Exception {
+        dfsCluster.shutdown();
+    }
 
-	/**
-	 * Runs the tests and collects their result in a TestResult.
-	 */
-	@Override
-	public void run(TestResult result) {
-		try {
-			int testCount = countTestCases();
-			for (int i = 0; i < testCount; i++) {
-				// cleanupStores();
-				Test each = this.testAt(i);
-				if (result.shouldStop())
-					break;
-				runTest(each, result);
-			}
-			tearDown();
-		} catch (Exception e) {
-			throw new IllegalStateException(e);
-		}
-	}
+    public void tearDown() throws Exception {
+        PregelixHyracksIntegrationUtil.deinit();
+        LOGGER.info("Hyracks mini-cluster shut down");
+        cleanupHDFS();
+    }
 
-	protected static List<String> getFileList(String ignorePath)
-			throws FileNotFoundException, IOException {
-		BufferedReader reader = new BufferedReader(new FileReader(ignorePath));
-		String s = null;
-		List<String> ignores = new ArrayList<String>();
-		while ((s = reader.readLine()) != null) {
-			ignores.add(s);
-		}
-		reader.close();
-		return ignores;
-	}
+    public static Test suite() throws Exception {
+        List<String> ignores = getFileList(PATH_TO_IGNORE);
+        List<String> onlys = getFileList(PATH_TO_ONLY);
+        File testData = new File(PATH_TO_JOBS);
+        File[] queries = testData.listFiles();
+        RunJobTestSuite testSuite = new RunJobTestSuite();
+        testSuite.setUp();
+        boolean onlyEnabled = false;
+        FileSystem dfs = FileSystem.get(testSuite.conf);
 
-	private static String jobExtToResExt(String fname) {
-		int dot = fname.lastIndexOf('.');
-		return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;
-	}
+        if (onlys.size() > 0) {
+            onlyEnabled = true;
+        }
+        for (File qFile : queries) {
+            if (isInList(ignores, qFile.getName()))
+                continue;
 
-	private static boolean isInList(List<String> onlys, String name) {
-		for (String only : onlys)
-			if (name.indexOf(only) >= 0)
-				return true;
-		return false;
-	}
+            if (qFile.isFile()) {
+                if (onlyEnabled && !isInList(onlys, qFile.getName())) {
+                    continue;
+                } else {
+                    String resultFileName = ACTUAL_RESULT_DIR + File.separator + jobExtToResExt(qFile.getName());
+                    String expectedFileName = EXPECTED_RESULT_DIR + File.separator + jobExtToResExt(qFile.getName());
+                    testSuite.addTest(new RunJobTestCase(HADOOP_CONF_PATH, qFile.getName(), qFile.getAbsolutePath()
+                            .toString(), resultFileName, expectedFileName, dfs));
+                }
+            }
+        }
+        return testSuite;
+    }
+
+    /**
+     * Runs the tests and collects their result in a TestResult.
+     */
+    @Override
+    public void run(TestResult result) {
+        try {
+            int testCount = countTestCases();
+            for (int i = 0; i < testCount; i++) {
+                // cleanupStores();
+                Test each = this.testAt(i);
+                if (result.shouldStop())
+                    break;
+                runTest(each, result);
+            }
+            tearDown();
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    protected static List<String> getFileList(String ignorePath) throws FileNotFoundException, IOException {
+        BufferedReader reader = new BufferedReader(new FileReader(ignorePath));
+        String s = null;
+        List<String> ignores = new ArrayList<String>();
+        while ((s = reader.readLine()) != null) {
+            ignores.add(s);
+        }
+        reader.close();
+        return ignores;
+    }
+
+    private static String jobExtToResExt(String fname) {
+        int dot = fname.lastIndexOf('.');
+        return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;
+    }
+
+    private static boolean isInList(List<String> onlys, String name) {
+        for (String only : onlys)
+            if (name.indexOf(only) >= 0)
+                return true;
+        return false;
+    }
 
 }
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestUtils.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestUtils.java
index d89ec46..d406125 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestUtils.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestUtils.java
@@ -20,6 +20,13 @@
 
 public class TestUtils {
 
+    public static void compareWithResultDir(File expectedFileDir, File actualFileDir) throws Exception {
+        String[] fileNames = expectedFileDir.list();
+        for (String fileName : fileNames) {
+            compareWithResult(new File(expectedFileDir, fileName), new File(actualFileDir, fileName));
+        }
+    }
+
     public static void compareWithResult(File expectedFile, File actualFile) throws Exception {
         BufferedReader readerExpected = new BufferedReader(new FileReader(expectedFile));
         BufferedReader readerActual = new BufferedReader(new FileReader(actualFile));
@@ -28,7 +35,6 @@
         try {
             while ((lineExpected = readerExpected.readLine()) != null) {
                 lineActual = readerActual.readLine();
-                // Assert.assertEquals(lineExpected, lineActual);
                 if (lineActual == null) {
                     throw new Exception("Actual result changed at line " + num + ":\n< " + lineExpected + "\n> ");
                 }
@@ -62,8 +68,10 @@
             if (row1.equals(row2))
                 continue;
 
-            String[] fields1 = row1.split(" ");
-            String[] fields2 = row2.split(" ");
+            boolean spaceOrTab = false;
+            spaceOrTab = row1.contains(" ");
+            String[] fields1 = spaceOrTab ? row1.split(" ") : row1.split("\t");
+            String[] fields2 = spaceOrTab ? row2.split(" ") : row2.split("\t");
 
             for (int j = 0; j < fields1.length; j++) {
                 if (fields1[j].equals(fields2[j])) {
@@ -76,7 +84,7 @@
                     float float1 = (float) double1.doubleValue();
                     float float2 = (float) double2.doubleValue();
 
-                    if (Math.abs(float1 - float2) == 0)
+                    if (Math.abs(float1 - float2) < 1.0e-7)
                         continue;
                     else {
                         return false;
diff --git a/pregelix/pregelix-example/src/test/resources/cluster/stores.properties b/pregelix/pregelix-example/src/test/resources/cluster/stores.properties
index daf881e..04732be 100644
--- a/pregelix/pregelix-example/src/test/resources/cluster/stores.properties
+++ b/pregelix/pregelix-example/src/test/resources/cluster/stores.properties
@@ -1 +1 @@
-store=teststore
\ No newline at end of file
+store=teststore1,teststore2
\ No newline at end of file
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result
deleted file mode 100644
index 45376e2..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result
+++ /dev/null
@@ -1,20 +0,0 @@
-0 0
-1 0
-2 0
-3 0
-4 0
-5 0
-6 0
-7 0
-8 0
-9 0
-10 0
-11 0
-12 0
-13 0
-14 0
-15 0
-16 0
-17 0
-18 0
-19 0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result/part-0
new file mode 100755
index 0000000..f1f1d9b
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result/part-0
@@ -0,0 +1,5 @@
+0	0
+4	0
+8	0
+12	0
+16	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result/part-1
new file mode 100755
index 0000000..0fa02c1
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result/part-1
@@ -0,0 +1,5 @@
+1	0
+5	0
+9	0
+13	0
+17	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result/part-2
new file mode 100755
index 0000000..542ccae
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result/part-2
@@ -0,0 +1,5 @@
+2	0
+6	0
+10	0
+14	0
+18	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result/part-3
new file mode 100755
index 0000000..1d5d6d9
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal.result/part-3
@@ -0,0 +1,5 @@
+3	0
+7	0
+11	0
+15	0
+19	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result
deleted file mode 100644
index dbc30fc..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result
+++ /dev/null
@@ -1,23 +0,0 @@
-0 0
-1 0
-2 0
-3 0
-4 0
-5 0
-6 0
-7 0
-8 0
-9 0
-10 0
-11 0
-12 0
-13 0
-14 0
-15 0
-16 0
-17 0
-18 0
-19 0
-21 21
-25 25
-27 27
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result/part-0
new file mode 100755
index 0000000..f1f1d9b
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result/part-0
@@ -0,0 +1,5 @@
+0	0
+4	0
+8	0
+12	0
+16	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result/part-1
new file mode 100755
index 0000000..4e7d87a
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result/part-1
@@ -0,0 +1,7 @@
+1	0
+5	0
+9	0
+13	0
+17	0
+21	21
+25	25
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result/part-2
new file mode 100755
index 0000000..542ccae
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result/part-2
@@ -0,0 +1,5 @@
+2	0
+6	0
+10	0
+14	0
+18	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result/part-3
new file mode 100755
index 0000000..513f3ff
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex.result/part-3
@@ -0,0 +1,6 @@
+3	0
+7	0
+11	0
+15	0
+19	0
+27	27
diff --git a/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result
deleted file mode 100644
index a30166c..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result
+++ /dev/null
@@ -1,13 +0,0 @@
-1 0.0
-5 0.0
-7 0.0
-11 0.0
-13 0.0
-17 0.0
-19 0.0
-100 0.0
-500 0.0
-700 0.0
-1100 0.0
-1300 0.0
-1700 0.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result/part-0
new file mode 100755
index 0000000..b5f7ed3
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result/part-0
@@ -0,0 +1,6 @@
+100	0.0
+500	0.0
+700	0.0
+1100	0.0
+1300	0.0
+1700	0.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result/part-1
new file mode 100755
index 0000000..4eca51d
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result/part-1
@@ -0,0 +1,4 @@
+1	0.0
+5	0.0
+13	0.0
+17	0.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result/part-2
new file mode 100755
index 0000000..e69de29
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result/part-2
diff --git a/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result/part-3
new file mode 100755
index 0000000..9446ff7
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation.result/part-3
@@ -0,0 +1,3 @@
+7	0.0
+11	0.0
+19	0.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result
deleted file mode 100644
index d238037..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result
+++ /dev/null
@@ -1,7 +0,0 @@
-1 1,2,3,4;
-2 2,3,4;
-3 
-4 
-5 
-6 
-7 
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result/part-0
new file mode 100755
index 0000000..902fadf
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result/part-0
@@ -0,0 +1 @@
+4	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result/part-1
new file mode 100755
index 0000000..ba34424
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result/part-1
@@ -0,0 +1,2 @@
+1	1,2,3,4;
+5	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result/part-2
new file mode 100755
index 0000000..834e389
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result/part-2
@@ -0,0 +1,2 @@
+2	2,3,4;
+6	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result/part-3
new file mode 100755
index 0000000..b8e2461
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique.result/part-3
@@ -0,0 +1,2 @@
+3	3,4;
+7	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2.result/part-0
new file mode 100755
index 0000000..902fadf
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2.result/part-0
@@ -0,0 +1 @@
+4	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2.result/part-1
new file mode 100755
index 0000000..b83e1a3
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2.result/part-1
@@ -0,0 +1,2 @@
+1	1,2,3;1,3,4;
+5	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2.result/part-2
new file mode 100755
index 0000000..45e2a23
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2.result/part-2
@@ -0,0 +1,2 @@
+2	2,3;
+6	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2.result/part-3
new file mode 100755
index 0000000..447aa38
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2.result/part-3
@@ -0,0 +1 @@
+3	3,4;3,5;
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3.result/part-0
new file mode 100755
index 0000000..b4dced4
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3.result/part-0
@@ -0,0 +1,5 @@
+0	0,19;
+4	4,5,6,7,8,9;
+8	
+12	12,13,14,15,16,17,18,19;
+16	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3.result/part-1
new file mode 100755
index 0000000..f554dfe
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3.result/part-1
@@ -0,0 +1,5 @@
+1	1,2,3,4,5,6,7,8,9;
+5	
+9	
+13	
+17	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3.result/part-2
new file mode 100755
index 0000000..4df9b8c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3.result/part-2
@@ -0,0 +1,5 @@
+2	2,3,4,5,6,7,8,9;
+6	
+10	
+14	
+18	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3.result/part-3
new file mode 100755
index 0000000..5131560
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3.result/part-3
@@ -0,0 +1,5 @@
+3	3,4,5,6,7,8,9;
+7	
+11	11,12,13,14,15,16,17,18,19;
+15	
+19	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRank.result b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result
deleted file mode 100644
index 9c4d83a..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRank.result
+++ /dev/null
@@ -1,20 +0,0 @@
-0 0.008290140026154316
-1 0.1535152819247165
-2 0.14646839195826475
-3 0.08125113985998214
-4 0.03976979906329426
-5 0.0225041581462058
-6 0.015736276824953852
-7 0.012542224114863661
-8 0.010628239626209894
-9 0.009294348455354817
-10 0.008290140026154316
-11 0.15351528192471647
-12 0.14646839195826472
-13 0.08125113985998214
-14 0.03976979906329425
-15 0.0225041581462058
-16 0.015736276824953852
-17 0.012542224114863661
-18 0.010628239626209894
-19 0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRank.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result/part-0
new file mode 100755
index 0000000..5b07add
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result/part-0
@@ -0,0 +1,10 @@
+0	0.008290140026154316
+4	0.039769799063294246
+8	0.010628239626209894
+12	0.14646839195826478
+16	0.015736276824953852
+20	0.008290140026154316
+24	0.039769799063294246
+28	0.010628239626209894
+32	0.14646839195826478
+36	0.015736276824953852
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRank.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result/part-1
new file mode 100755
index 0000000..caf8a4c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result/part-1
@@ -0,0 +1,10 @@
+1	0.15351528192471647
+5	0.0225041581462058
+9	0.009294348455354817
+13	0.08125113985998214
+17	0.012542224114863661
+21	0.15351528192471647
+25	0.0225041581462058
+29	0.009294348455354817
+33	0.08125113985998214
+37	0.012542224114863661
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRank.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result/part-2
new file mode 100755
index 0000000..29a9d52
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result/part-2
@@ -0,0 +1,10 @@
+2	0.14646839195826475
+6	0.015736276824953852
+10	0.008290140026154316
+14	0.03976979906329426
+18	0.010628239626209894
+22	0.14646839195826472
+26	0.01573627682495385
+30	0.008290140026154316
+34	0.03976979906329426
+38	0.010628239626209894
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRank.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result/part-3
new file mode 100755
index 0000000..c9bfeb8
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRank.result/part-3
@@ -0,0 +1,10 @@
+3	0.08125113985998214
+7	0.012542224114863663
+11	0.1535152819247165
+15	0.0225041581462058
+19	0.009294348455354817
+23	0.08125113985998211
+27	0.012542224114863661
+31	0.1535152819247165
+35	0.0225041581462058
+39	0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result
deleted file mode 100644
index 6432eda..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result
+++ /dev/null
@@ -1,20 +0,0 @@
-0 0.008290140026154316
-1 0.1535152819247165
-2 0.14646839195826475
-3 0.08125113985998214
-4 0.03976979906329426
-5 0.0225041581462058
-6 0.015736276824953852
-7 0.012542224114863661
-8 0.010628239626209894
-9 0.009294348455354817
-10 0.008290140026154316
-11 0.15351528192471647
-12 0.14646839195826472
-13 0.08125113985998214
-14 0.03976979906329426
-15 0.0225041581462058
-16 0.015736276824953852
-17 0.012542224114863661
-18 0.010628239626209894
-19 0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result/part-0
new file mode 100755
index 0000000..383076e
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result/part-0
@@ -0,0 +1,5 @@
+0	0.008290140026154316
+4	0.03976979906329426
+8	0.010628239626209894
+12	0.14646839195826478
+16	0.015736276824953852
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result/part-1
new file mode 100755
index 0000000..0b1a38c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result/part-1
@@ -0,0 +1,5 @@
+1	0.15351528192471653
+5	0.0225041581462058
+9	0.009294348455354817
+13	0.08125113985998214
+17	0.012542224114863661
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result/part-2
new file mode 100755
index 0000000..ec995b2
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result/part-2
@@ -0,0 +1,5 @@
+2	0.14646839195826478
+6	0.015736276824953852
+10	0.008290140026154316
+14	0.03976979906329426
+18	0.010628239626209894
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result/part-3
new file mode 100755
index 0000000..edb7484
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal.result/part-3
@@ -0,0 +1,5 @@
+3	0.08125113985998214
+7	0.012542224114863661
+11	0.15351528192471653
+15	0.0225041581462058
+19	0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result
deleted file mode 100644
index 2bd09e1..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result
+++ /dev/null
@@ -1,23 +0,0 @@
-0 0.0072088164890121405
-1 0.12352056961948686
-2 0.12045670441668178
-3 0.06798545786459467
-4 0.03387281259892814
-5 0.01942600635480669
-6 0.013661020012182747
-7 0.0109034351563503
-8 0.009241684574402657
-9 0.008082028259564783
-10 0.007208817414047232
-11 0.07555839219845861
-12 0.07249452699565352
-13 0.05063539695954156
-14 0.029644452692487822
-15 0.018670183493927354
-16 0.013558283213067561
-17 0.010892790899883237
-18 0.009240874593661061
-19 0.008081987856433137
-21 0.006521739130434782
-25 0.006521739130434782
-27 0.006521739130434782
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result/part-0
new file mode 100755
index 0000000..1d26aee
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result/part-0
@@ -0,0 +1,5 @@
+0	0.0072088164890121405
+4	0.03387281259892814
+8	0.009241684574402657
+12	0.07249452699565351
+16	0.013558283213067561
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result/part-1
new file mode 100755
index 0000000..d013c3e
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result/part-1
@@ -0,0 +1,7 @@
+1	0.12352056961948689
+5	0.01942600635480669
+9	0.008082028259564783
+13	0.050635396959541557
+17	0.010892790899883237
+21	0.006521739130434782
+25	0.006521739130434782
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result/part-2
new file mode 100755
index 0000000..e8aa0e1
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result/part-2
@@ -0,0 +1,5 @@
+2	0.12045670441668178
+6	0.013661020012182747
+10	0.007208817414047232
+14	0.029644452692487822
+18	0.009240874593661061
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result/part-3
new file mode 100755
index 0000000..e52970d
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex.result/part-3
@@ -0,0 +1,6 @@
+3	0.06798545786459467
+7	0.0109034351563503
+11	0.0755583921984586
+15	0.018670183493927354
+19	0.008081987856433137
+27	0.006521739130434782
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result
deleted file mode 100644
index 6432eda..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result
+++ /dev/null
@@ -1,20 +0,0 @@
-0 0.008290140026154316
-1 0.1535152819247165
-2 0.14646839195826475
-3 0.08125113985998214
-4 0.03976979906329426
-5 0.0225041581462058
-6 0.015736276824953852
-7 0.012542224114863661
-8 0.010628239626209894
-9 0.009294348455354817
-10 0.008290140026154316
-11 0.15351528192471647
-12 0.14646839195826472
-13 0.08125113985998214
-14 0.03976979906329426
-15 0.0225041581462058
-16 0.015736276824953852
-17 0.012542224114863661
-18 0.010628239626209894
-19 0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result/part-0
new file mode 100755
index 0000000..383076e
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result/part-0
@@ -0,0 +1,5 @@
+0	0.008290140026154316
+4	0.03976979906329426
+8	0.010628239626209894
+12	0.14646839195826478
+16	0.015736276824953852
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result/part-1
new file mode 100755
index 0000000..0b1a38c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result/part-1
@@ -0,0 +1,5 @@
+1	0.15351528192471653
+5	0.0225041581462058
+9	0.009294348455354817
+13	0.08125113985998214
+17	0.012542224114863661
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result/part-2
new file mode 100755
index 0000000..ec995b2
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result/part-2
@@ -0,0 +1,5 @@
+2	0.14646839195826478
+6	0.015736276824953852
+10	0.008290140026154316
+14	0.03976979906329426
+18	0.010628239626209894
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result/part-3
new file mode 100755
index 0000000..5593738
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic.result/part-3
@@ -0,0 +1,5 @@
+3	0.08125113985998214
+7	0.012542224114863661
+11	0.15351528192471653
+15	0.022504158146205808
+19	0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result
deleted file mode 100755
index 9a747a6..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result
+++ /dev/null
@@ -1,20 +0,0 @@
-0 0.008290140026154316
-1 0.15351528192471647
-2 0.14646839195826475
-3 0.08125113985998211
-4 0.03976979906329425
-5 0.0225041581462058
-6 0.01573627682495385
-7 0.012542224114863661
-8 0.010628239626209894
-9 0.009294348455354817
-10 0.008290140026154316
-11 0.1535152819247165
-12 0.14646839195826475
-13 0.08125113985998214
-14 0.03976979906329426
-15 0.0225041581462058
-16 0.015736276824953852
-17 0.012542224114863661
-18 0.010628239626209894
-19 0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result/part-0
new file mode 100755
index 0000000..6fd5f60
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result/part-0
@@ -0,0 +1,5 @@
+0	0.008290140026154316
+4	0.03976979906329425
+8	0.010628239626209894
+12	0.14646839195826475
+16	0.01573627682495385
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result/part-1
new file mode 100755
index 0000000..0b1a38c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result/part-1
@@ -0,0 +1,5 @@
+1	0.15351528192471653
+5	0.0225041581462058
+9	0.009294348455354817
+13	0.08125113985998214
+17	0.012542224114863661
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result/part-2
new file mode 100755
index 0000000..69a2803
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result/part-2
@@ -0,0 +1,5 @@
+2	0.14646839195826478
+6	0.015736276824953852
+10	0.008290140026154316
+14	0.03976979906329425
+18	0.010628239626209894
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result/part-3
new file mode 100755
index 0000000..7dcb359
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner.result/part-3
@@ -0,0 +1,5 @@
+3	0.08125113985998214
+7	0.012542224114863661
+11	0.1535152819247165
+15	0.0225041581462058
+19	0.009294348455354817
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result
deleted file mode 100644
index a1dfc0f..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result
+++ /dev/null
@@ -1,23 +0,0 @@
-0 2
-1 3
-2 1
-3 1
-4 1
-5 1
-6 1
-7 1
-8 1
-9 1
-10 3
-11 2
-12 2
-13 2
-14 2
-15 2
-16 2
-17 2
-18 2
-19 2
-21 0
-25 0
-27 0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result/part-0
new file mode 100755
index 0000000..6e3ba89
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result/part-0
@@ -0,0 +1,5 @@
+0	0
+4	1
+8	1
+12	2
+16	2
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result/part-1
new file mode 100755
index 0000000..2fcc8eb
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result/part-1
@@ -0,0 +1,7 @@
+1	1
+5	1
+9	1
+13	2
+17	2
+21	0
+25	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result/part-2
new file mode 100755
index 0000000..c93732c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result/part-2
@@ -0,0 +1,5 @@
+2	1
+6	1
+10	3
+14	2
+18	2
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result/part-3
new file mode 100755
index 0000000..c745349
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex.result/part-3
@@ -0,0 +1,6 @@
+3	1
+7	1
+11	2
+15	2
+19	2
+27	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result
deleted file mode 100644
index 1693fb2..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result
+++ /dev/null
@@ -1,23 +0,0 @@
-0 1
-1 1
-2 1
-3 1
-4 1
-5 1
-6 1
-7 1
-8 1
-9 1
-10 1
-11 1
-12 1
-13 1
-14 1
-15 1
-16 1
-17 1
-18 1
-19 1
-21 0
-25 2
-27 0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result/part-0
new file mode 100755
index 0000000..8e02e13
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result/part-0
@@ -0,0 +1,5 @@
+0	1
+4	1
+8	1
+12	1
+16	1
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result/part-1
new file mode 100755
index 0000000..cfbb359
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result/part-1
@@ -0,0 +1,7 @@
+1	1
+5	1
+9	1
+13	1
+17	1
+21	0
+25	2
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result/part-2
new file mode 100755
index 0000000..8e3ca7c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result/part-2
@@ -0,0 +1,5 @@
+2	1
+6	1
+10	1
+14	1
+18	1
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result/part-3
new file mode 100755
index 0000000..5f83a3d
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity.result/part-3
@@ -0,0 +1,6 @@
+3	1
+7	1
+11	1
+15	1
+19	1
+27	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result
deleted file mode 100644
index 46d1c73..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result
+++ /dev/null
@@ -1,20 +0,0 @@
-0 0.0
-1 0.0
-2 100.0
-3 300.0
-4 600.0
-5 1000.0
-6 1500.0
-7 2100.0
-8 2800.0
-9 3600.0
-10 4500.0
-11 5500.0
-12 6600.0
-13 7800.0
-14 9100.0
-15 10500.0
-16 12000.0
-17 13600.0
-18 15300.0
-19 17100.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result/part-0
new file mode 100755
index 0000000..e8eadcd
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result/part-0
@@ -0,0 +1,10 @@
+0	0.0
+4	600.0
+8	2800.0
+12	6600.0
+16	12000.0
+20	19000.0
+24	27600.0
+28	37800.0
+32	49600.0
+36	63000.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result/part-1
new file mode 100755
index 0000000..c3d200b
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result/part-1
@@ -0,0 +1,10 @@
+1	0.0
+5	1000.0
+9	3600.0
+13	7800.0
+17	13600.0
+21	21000.0
+25	30000.0
+29	40600.0
+33	52800.0
+37	66600.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result/part-2
new file mode 100755
index 0000000..779a683
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result/part-2
@@ -0,0 +1,10 @@
+2	100.0
+6	1500.0
+10	4500.0
+14	9100.0
+18	15300.0
+22	23100.0
+26	32500.0
+30	43500.0
+34	56100.0
+38	70300.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result/part-3
new file mode 100755
index 0000000..57d26c9
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths.result/part-3
@@ -0,0 +1,10 @@
+3	300.0
+7	2100.0
+11	5500.0
+15	10500.0
+19	17100.0
+23	25300.0
+27	35100.0
+31	46500.0
+35	59500.0
+39	74100.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result
deleted file mode 100644
index b42462f..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result
+++ /dev/null
@@ -1,20 +0,0 @@
-0 0.0
-1 1.0
-2 2.0
-3 3.0
-4 4.0
-5 5.0
-6 6.0
-7 7.0
-8 8.0
-9 9.0
-10 10.0
-11 11.0
-12 12.0
-13 13.0
-14 14.0
-15 15.0
-16 16.0
-17 17.0
-18 18.0
-19 19.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result/part-0
new file mode 100755
index 0000000..a1cabe2
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result/part-0
@@ -0,0 +1,5 @@
+0	0.0
+4	4.0
+8	8.0
+12	12.0
+16	16.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result/part-1
new file mode 100755
index 0000000..303ed2a
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result/part-1
@@ -0,0 +1,5 @@
+1	1.0
+5	5.0
+9	9.0
+13	13.0
+17	17.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result/part-2
new file mode 100755
index 0000000..0020d25
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result/part-2
@@ -0,0 +1,5 @@
+2	2.0
+6	6.0
+10	10.0
+14	14.0
+18	18.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result/part-3
new file mode 100755
index 0000000..e2657bd
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal.result/part-3
@@ -0,0 +1,5 @@
+3	3.0
+7	7.0
+11	11.0
+15	15.0
+19	19.0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result
deleted file mode 100644
index 4818e13..0000000
--- a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result
+++ /dev/null
@@ -1,7 +0,0 @@
-1 3
-2 2
-3 0
-4 0
-5 1
-6 0
-7 0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/.part-0.crc b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/.part-0.crc
new file mode 100644
index 0000000..61e7df3
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/.part-0.crc
Binary files differ
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/.part-1.crc b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/.part-1.crc
new file mode 100644
index 0000000..9f34827
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/.part-1.crc
Binary files differ
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/.part-2.crc b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/.part-2.crc
new file mode 100644
index 0000000..cfe7fa2
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/.part-2.crc
Binary files differ
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/.part-3.crc b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/.part-3.crc
new file mode 100644
index 0000000..9959654
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/.part-3.crc
Binary files differ
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/part-0 b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/part-0
new file mode 100755
index 0000000..b36d2b5
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/part-0
@@ -0,0 +1 @@
+4	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/part-1 b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/part-1
new file mode 100755
index 0000000..2812c76
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/part-1
@@ -0,0 +1,2 @@
+1	3
+5	1
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/part-2 b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/part-2
new file mode 100755
index 0000000..25cfb7b
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/part-2
@@ -0,0 +1,2 @@
+2	2
+6	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/part-3 b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/part-3
new file mode 100755
index 0000000..2bd8d38
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting.result/part-3
@@ -0,0 +1,2 @@
+3	0
+7	0
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml
new file mode 100644
index 0000000..5621259
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml
@@ -0,0 +1,142 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
+<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
+<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
+<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
+<property><name>mapred.jobtracker.instrumentation</name><value>org.apache.hadoop.mapred.JobTrackerMetricsInst</value></property>
+<property><name>mapred.skip.reduce.auto.incr.proc.count</name><value>true</value></property>
+<property><name>fs.hsftp.impl</name><value>org.apache.hadoop.hdfs.HsftpFileSystem</value></property>
+<property><name>mapred.input.dir</name><value>file:/clique2</value></property>
+<property><name>mapred.submit.replication</name><value>10</value></property>
+<property><name>ipc.server.tcpnodelay</name><value>false</value></property>
+<property><name>fs.checkpoint.dir</name><value>${hadoop.tmp.dir}/dfs/namesecondary</value></property>
+<property><name>mapred.output.compression.type</name><value>RECORD</value></property>
+<property><name>mapred.job.shuffle.merge.percent</name><value>0.66</value></property>
+<property><name>mapred.child.java.opts</name><value>-Xmx200m</value></property>
+<property><name>mapred.queue.default.acl-submit-job</name><value>*</value></property>
+<property><name>keep.failed.task.files</name><value>false</value></property>
+<property><name>mapred.jobtracker.job.history.block.size</name><value>3145728</value></property>
+<property><name>io.bytes.per.checksum</name><value>512</value></property>
+<property><name>mapred.task.tracker.report.address</name><value>127.0.0.1:0</value></property>
+<property><name>hadoop.util.hash.type</name><value>murmur</value></property>
+<property><name>fs.hdfs.impl</name><value>org.apache.hadoop.hdfs.DistributedFileSystem</value></property>
+<property><name>fs.ramfs.impl</name><value>org.apache.hadoop.fs.InMemoryFileSystem</value></property>
+<property><name>mapred.jobtracker.restart.recover</name><value>false</value></property>
+<property><name>fs.hftp.impl</name><value>org.apache.hadoop.hdfs.HftpFileSystem</value></property>
+<property><name>fs.checkpoint.period</name><value>3600</value></property>
+<property><name>mapred.child.tmp</name><value>./tmp</value></property>
+<property><name>mapred.local.dir.minspacekill</name><value>0</value></property>
+<property><name>map.sort.class</name><value>org.apache.hadoop.util.QuickSort</value></property>
+<property><name>hadoop.logfile.count</name><value>10</value></property>
+<property><name>ipc.client.connection.maxidletime</name><value>10000</value></property>
+<property><name>mapred.output.dir</name><value>/resultclique</value></property>
+<property><name>io.map.index.skip</name><value>0</value></property>
+<property><name>mapred.tasktracker.expiry.interval</name><value>600000</value></property>
+<property><name>mapred.output.compress</name><value>false</value></property>
+<property><name>io.seqfile.lazydecompress</name><value>true</value></property>
+<property><name>mapred.reduce.parallel.copies</name><value>5</value></property>
+<property><name>fs.checkpoint.size</name><value>67108864</value></property>
+<property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
+<property><name>mapred.job.name</name><value>Maximal Clique 2</value></property>
+<property><name>local.cache.size</name><value>10737418240</value></property>
+<property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
+<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
+<property><name>fs.file.impl</name><value>org.apache.hadoop.fs.LocalFileSystem</value></property>
+<property><name>mapred.task.tracker.http.address</name><value>0.0.0.0:50060</value></property>
+<property><name>mapred.task.timeout</name><value>600000</value></property>
+<property><name>fs.kfs.impl</name><value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value></property>
+<property><name>mapred.max.tracker.blacklists</name><value>4</value></property>
+<property><name>fs.s3.buffer.dir</name><value>${hadoop.tmp.dir}/s3</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.dir</name><value>/jobtracker/jobsInfo</value></property>
+<property><name>ipc.client.kill.max</name><value>10</value></property>
+<property><name>mapred.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property>
+<property><name>mapred.reduce.tasks.speculative.execution</name><value>true</value></property>
+<property><name>io.sort.record.percent</name><value>0.05</value></property>
+<property><name>hadoop.security.authorization</name><value>false</value></property>
+<property><name>mapred.max.tracker.failures</name><value>4</value></property>
+<property><name>mapred.jobtracker.taskScheduler</name><value>org.apache.hadoop.mapred.JobQueueTaskScheduler</value></property>
+<property><name>mapred.tasktracker.dns.interface</name><value>default</value></property>
+<property><name>mapred.map.tasks</name><value>2</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.hours</name><value>0</value></property>
+<property><name>fs.s3.sleepTimeSeconds</name><value>10</value></property>
+<property><name>fs.default.name</name><value>file:///</value></property>
+<property><name>tasktracker.http.threads</name><value>40</value></property>
+<property><name>mapred.tasktracker.taskmemorymanager.monitoring-interval</name><value>5000</value></property>
+<property><name>hadoop.rpc.socket.factory.class.default</name><value>org.apache.hadoop.net.StandardSocketFactory</value></property>
+<property><name>mapred.reduce.tasks</name><value>1</value></property>
+<property><name>topology.node.switch.mapping.impl</name><value>org.apache.hadoop.net.ScriptBasedMapping</value></property>
+<property><name>pregelix.vertexClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex</value></property>
+<property><name>mapred.skip.reduce.max.skip.groups</name><value>0</value></property>
+<property><name>io.file.buffer.size</name><value>4096</value></property>
+<property><name>mapred.jobtracker.maxtasks.per.job</name><value>-1</value></property>
+<property><name>mapred.tasktracker.indexcache.mb</name><value>10</value></property>
+<property><name>mapred.tasktracker.map.tasks.maximum</name><value>2</value></property>
+<property><name>fs.har.impl.disable.cache</name><value>true</value></property>
+<property><name>mapred.task.profile.maps</name><value>0-2</value></property>
+<property><name>hadoop.native.lib</name><value>true</value></property>
+<property><name>fs.s3.block.size</name><value>67108864</value></property>
+<property><name>mapred.job.reuse.jvm.num.tasks</name><value>1</value></property>
+<property><name>mapred.job.tracker.http.address</name><value>0.0.0.0:50030</value></property>
+<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
+<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
+<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
+<property><name>mapred.queue.names</name><value>default</value></property>
+<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
+<property><name>io.mapfile.bloom.error.rate</name><value>0.005</value></property>
+<property><name>mapred.job.tracker</name><value>local</value></property>
+<property><name>io.skip.checksum.errors</name><value>false</value></property>
+<property><name>mapred.reduce.max.attempts</name><value>4</value></property>
+<property><name>fs.s3.maxRetries</name><value>4</value></property>
+<property><name>ipc.server.listen.queue.size</name><value>128</value></property>
+<property><name>fs.trash.interval</name><value>0</value></property>
+<property><name>mapred.local.dir.minspacestart</name><value>0</value></property>
+<property><name>fs.s3.impl</name><value>org.apache.hadoop.fs.s3.S3FileSystem</value></property>
+<property><name>io.seqfile.sorter.recordlimit</name><value>1000000</value></property>
+<property><name>io.mapfile.bloom.size</name><value>1048576</value></property>
+<property><name>io.sort.mb</name><value>100</value></property>
+<property><name>mapred.local.dir</name><value>${hadoop.tmp.dir}/mapred/local</value></property>
+<property><name>io.sort.factor</name><value>10</value></property>
+<property><name>mapred.task.profile</name><value>false</value></property>
+<property><name>job.end.retry.interval</name><value>30000</value></property>
+<property><name>mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill</name><value>5000</value></property>
+<property><name>mapred.jobtracker.completeuserjobs.maximum</name><value>100</value></property>
+<property><name>mapred.task.profile.reduces</name><value>0-2</value></property>
+<property><name>webinterface.private.actions</name><value>false</value></property>
+<property><name>hadoop.tmp.dir</name><value>/tmp/hadoop-${user.name}</value></property>
+<property><name>mapred.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.skip.attempts.to.start.skipping</name><value>2</value></property>
+<property><name>mapred.temp.dir</name><value>${hadoop.tmp.dir}/mapred/temp</value></property>
+<property><name>mapred.merge.recordsBeforeProgress</name><value>10000</value></property>
+<property><name>mapred.map.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.compress.map.output</name><value>false</value></property>
+<property><name>io.sort.spill.percent</name><value>0.80</value></property>
+<property><name>fs.checkpoint.edits.dir</name><value>${fs.checkpoint.dir}</value></property>
+<property><name>mapred.userlog.retain.hours</name><value>24</value></property>
+<property><name>mapred.system.dir</name><value>${hadoop.tmp.dir}/mapred/system</value></property>
+<property><name>mapred.line.input.format.linespermap</name><value>1</value></property>
+<property><name>job.end.retry.attempts</name><value>0</value></property>
+<property><name>ipc.client.idlethreshold</name><value>4000</value></property>
+<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex$MaximalCliqueVertexOutputFormat</value></property>
+<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
+<property><name>hadoop.logfile.size</name><value>10000000</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.maximalclique.TextMaximalCliqueInputFormat</value></property>
+<property><name>pregelix.aggregatorClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueAggregator</value></property>
+<property><name>mapred.job.queue.name</name><value>default</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>pregelix.incStateLength</name><value>true</value></property>
+<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
+<property><name>topology.script.number.args</name><value>100</value></property>
+<property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
+<property><name>fs.ftp.impl</name><value>org.apache.hadoop.fs.ftp.FTPFileSystem</value></property>
+<property><name>mapred.task.cache.levels</name><value>2</value></property>
+<property><name>mapred.job.tracker.handler.count</name><value>10</value></property>
+<property><name>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization</value></property>
+<property><name>ipc.client.connect.max.retries</name><value>10</value></property>
+<property><name>mapred.min.split.size</name><value>0</value></property>
+<property><name>mapred.map.max.attempts</name><value>4</value></property>
+<property><name>jobclient.output.filter</name><value>FAILED</value></property>
+<property><name>ipc.client.tcpnodelay</name><value>false</value></property>
+<property><name>mapred.acls.enabled</name><value>false</value></property>
+</configuration>
\ No newline at end of file
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml
new file mode 100644
index 0000000..d4f81ba
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml
@@ -0,0 +1,142 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
+<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
+<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
+<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
+<property><name>mapred.jobtracker.instrumentation</name><value>org.apache.hadoop.mapred.JobTrackerMetricsInst</value></property>
+<property><name>mapred.skip.reduce.auto.incr.proc.count</name><value>true</value></property>
+<property><name>fs.hsftp.impl</name><value>org.apache.hadoop.hdfs.HsftpFileSystem</value></property>
+<property><name>mapred.input.dir</name><value>file:/clique3</value></property>
+<property><name>mapred.submit.replication</name><value>10</value></property>
+<property><name>ipc.server.tcpnodelay</name><value>false</value></property>
+<property><name>fs.checkpoint.dir</name><value>${hadoop.tmp.dir}/dfs/namesecondary</value></property>
+<property><name>mapred.output.compression.type</name><value>RECORD</value></property>
+<property><name>mapred.job.shuffle.merge.percent</name><value>0.66</value></property>
+<property><name>mapred.child.java.opts</name><value>-Xmx200m</value></property>
+<property><name>mapred.queue.default.acl-submit-job</name><value>*</value></property>
+<property><name>keep.failed.task.files</name><value>false</value></property>
+<property><name>mapred.jobtracker.job.history.block.size</name><value>3145728</value></property>
+<property><name>io.bytes.per.checksum</name><value>512</value></property>
+<property><name>mapred.task.tracker.report.address</name><value>127.0.0.1:0</value></property>
+<property><name>hadoop.util.hash.type</name><value>murmur</value></property>
+<property><name>fs.hdfs.impl</name><value>org.apache.hadoop.hdfs.DistributedFileSystem</value></property>
+<property><name>fs.ramfs.impl</name><value>org.apache.hadoop.fs.InMemoryFileSystem</value></property>
+<property><name>mapred.jobtracker.restart.recover</name><value>false</value></property>
+<property><name>fs.hftp.impl</name><value>org.apache.hadoop.hdfs.HftpFileSystem</value></property>
+<property><name>fs.checkpoint.period</name><value>3600</value></property>
+<property><name>mapred.child.tmp</name><value>./tmp</value></property>
+<property><name>mapred.local.dir.minspacekill</name><value>0</value></property>
+<property><name>map.sort.class</name><value>org.apache.hadoop.util.QuickSort</value></property>
+<property><name>hadoop.logfile.count</name><value>10</value></property>
+<property><name>ipc.client.connection.maxidletime</name><value>10000</value></property>
+<property><name>mapred.output.dir</name><value>/resultclique</value></property>
+<property><name>io.map.index.skip</name><value>0</value></property>
+<property><name>mapred.tasktracker.expiry.interval</name><value>600000</value></property>
+<property><name>mapred.output.compress</name><value>false</value></property>
+<property><name>io.seqfile.lazydecompress</name><value>true</value></property>
+<property><name>mapred.reduce.parallel.copies</name><value>5</value></property>
+<property><name>fs.checkpoint.size</name><value>67108864</value></property>
+<property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
+<property><name>mapred.job.name</name><value>Maximal Clique 3</value></property>
+<property><name>local.cache.size</name><value>10737418240</value></property>
+<property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
+<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
+<property><name>fs.file.impl</name><value>org.apache.hadoop.fs.LocalFileSystem</value></property>
+<property><name>mapred.task.tracker.http.address</name><value>0.0.0.0:50060</value></property>
+<property><name>mapred.task.timeout</name><value>600000</value></property>
+<property><name>fs.kfs.impl</name><value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value></property>
+<property><name>mapred.max.tracker.blacklists</name><value>4</value></property>
+<property><name>fs.s3.buffer.dir</name><value>${hadoop.tmp.dir}/s3</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.dir</name><value>/jobtracker/jobsInfo</value></property>
+<property><name>ipc.client.kill.max</name><value>10</value></property>
+<property><name>mapred.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property>
+<property><name>mapred.reduce.tasks.speculative.execution</name><value>true</value></property>
+<property><name>io.sort.record.percent</name><value>0.05</value></property>
+<property><name>hadoop.security.authorization</name><value>false</value></property>
+<property><name>mapred.max.tracker.failures</name><value>4</value></property>
+<property><name>mapred.jobtracker.taskScheduler</name><value>org.apache.hadoop.mapred.JobQueueTaskScheduler</value></property>
+<property><name>mapred.tasktracker.dns.interface</name><value>default</value></property>
+<property><name>mapred.map.tasks</name><value>2</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.hours</name><value>0</value></property>
+<property><name>fs.s3.sleepTimeSeconds</name><value>10</value></property>
+<property><name>fs.default.name</name><value>file:///</value></property>
+<property><name>tasktracker.http.threads</name><value>40</value></property>
+<property><name>mapred.tasktracker.taskmemorymanager.monitoring-interval</name><value>5000</value></property>
+<property><name>hadoop.rpc.socket.factory.class.default</name><value>org.apache.hadoop.net.StandardSocketFactory</value></property>
+<property><name>mapred.reduce.tasks</name><value>1</value></property>
+<property><name>topology.node.switch.mapping.impl</name><value>org.apache.hadoop.net.ScriptBasedMapping</value></property>
+<property><name>pregelix.vertexClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex</value></property>
+<property><name>mapred.skip.reduce.max.skip.groups</name><value>0</value></property>
+<property><name>io.file.buffer.size</name><value>4096</value></property>
+<property><name>mapred.jobtracker.maxtasks.per.job</name><value>-1</value></property>
+<property><name>mapred.tasktracker.indexcache.mb</name><value>10</value></property>
+<property><name>mapred.tasktracker.map.tasks.maximum</name><value>2</value></property>
+<property><name>fs.har.impl.disable.cache</name><value>true</value></property>
+<property><name>mapred.task.profile.maps</name><value>0-2</value></property>
+<property><name>hadoop.native.lib</name><value>true</value></property>
+<property><name>fs.s3.block.size</name><value>67108864</value></property>
+<property><name>mapred.job.reuse.jvm.num.tasks</name><value>1</value></property>
+<property><name>mapred.job.tracker.http.address</name><value>0.0.0.0:50030</value></property>
+<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
+<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
+<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
+<property><name>mapred.queue.names</name><value>default</value></property>
+<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
+<property><name>io.mapfile.bloom.error.rate</name><value>0.005</value></property>
+<property><name>mapred.job.tracker</name><value>local</value></property>
+<property><name>io.skip.checksum.errors</name><value>false</value></property>
+<property><name>mapred.reduce.max.attempts</name><value>4</value></property>
+<property><name>fs.s3.maxRetries</name><value>4</value></property>
+<property><name>ipc.server.listen.queue.size</name><value>128</value></property>
+<property><name>fs.trash.interval</name><value>0</value></property>
+<property><name>mapred.local.dir.minspacestart</name><value>0</value></property>
+<property><name>fs.s3.impl</name><value>org.apache.hadoop.fs.s3.S3FileSystem</value></property>
+<property><name>io.seqfile.sorter.recordlimit</name><value>1000000</value></property>
+<property><name>io.mapfile.bloom.size</name><value>1048576</value></property>
+<property><name>io.sort.mb</name><value>100</value></property>
+<property><name>mapred.local.dir</name><value>${hadoop.tmp.dir}/mapred/local</value></property>
+<property><name>io.sort.factor</name><value>10</value></property>
+<property><name>mapred.task.profile</name><value>false</value></property>
+<property><name>job.end.retry.interval</name><value>30000</value></property>
+<property><name>mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill</name><value>5000</value></property>
+<property><name>mapred.jobtracker.completeuserjobs.maximum</name><value>100</value></property>
+<property><name>mapred.task.profile.reduces</name><value>0-2</value></property>
+<property><name>webinterface.private.actions</name><value>false</value></property>
+<property><name>hadoop.tmp.dir</name><value>/tmp/hadoop-${user.name}</value></property>
+<property><name>mapred.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.skip.attempts.to.start.skipping</name><value>2</value></property>
+<property><name>mapred.temp.dir</name><value>${hadoop.tmp.dir}/mapred/temp</value></property>
+<property><name>mapred.merge.recordsBeforeProgress</name><value>10000</value></property>
+<property><name>mapred.map.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.compress.map.output</name><value>false</value></property>
+<property><name>io.sort.spill.percent</name><value>0.80</value></property>
+<property><name>fs.checkpoint.edits.dir</name><value>${fs.checkpoint.dir}</value></property>
+<property><name>mapred.userlog.retain.hours</name><value>24</value></property>
+<property><name>mapred.system.dir</name><value>${hadoop.tmp.dir}/mapred/system</value></property>
+<property><name>mapred.line.input.format.linespermap</name><value>1</value></property>
+<property><name>job.end.retry.attempts</name><value>0</value></property>
+<property><name>ipc.client.idlethreshold</name><value>4000</value></property>
+<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueVertex$MaximalCliqueVertexOutputFormat</value></property>
+<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
+<property><name>hadoop.logfile.size</name><value>10000000</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.maximalclique.TextMaximalCliqueInputFormat</value></property>
+<property><name>pregelix.aggregatorClass</name><value>edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueAggregator</value></property>
+<property><name>mapred.job.queue.name</name><value>default</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>pregelix.incStateLength</name><value>true</value></property>
+<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
+<property><name>topology.script.number.args</name><value>100</value></property>
+<property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
+<property><name>fs.ftp.impl</name><value>org.apache.hadoop.fs.ftp.FTPFileSystem</value></property>
+<property><name>mapred.task.cache.levels</name><value>2</value></property>
+<property><name>mapred.job.tracker.handler.count</name><value>10</value></property>
+<property><name>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization</value></property>
+<property><name>ipc.client.connect.max.retries</name><value>10</value></property>
+<property><name>mapred.min.split.size</name><value>0</value></property>
+<property><name>mapred.map.max.attempts</name><value>4</value></property>
+<property><name>jobclient.output.filter</name><value>FAILED</value></property>
+<property><name>ipc.client.tcpnodelay</name><value>false</value></property>
+<property><name>mapred.acls.enabled</name><value>false</value></property>
+</configuration>
\ No newline at end of file
diff --git a/pregelix/pregelix-runtime/pom.xml b/pregelix/pregelix-runtime/pom.xml
index bce7b12..29b6ba7 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
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
index e3c5a48..b6df213 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
@@ -109,7 +109,7 @@
             @Override
             public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writers)
                     throws HyracksDataException {
-                this.conf = confFactory.createConfiguration();
+                this.conf = confFactory.createConfiguration(ctx);
                 this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf);
                 this.aggregator = BspUtils.createGlobalAggregator(conf);
                 this.aggregator.init();
@@ -175,8 +175,12 @@
                 ArrayListWritable msgContentList = (ArrayListWritable) tuple[1];
                 msgContentList.reset(msgIterator);
 
-                if (!msgIterator.hasNext() && vertex.isHalted())
+                if (!msgIterator.hasNext() && vertex.isHalted()) {
                     return;
+                }
+                if (vertex.isHalted()) {
+                    vertex.activate();
+                }
 
                 try {
                 	vertex.activate();
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
index 0cf64a0..9998205 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
@@ -112,7 +112,7 @@
             @Override
             public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writers)
                     throws HyracksDataException {
-                this.conf = confFactory.createConfiguration();
+                this.conf = confFactory.createConfiguration(ctx);
                 this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf);
                 this.aggregator = BspUtils.createGlobalAggregator(conf);
                 this.aggregator.init();
@@ -176,8 +176,12 @@
                 vertex.setOutputAppenders(appenders);
                 vertex.setOutputTupleBuilders(tbs);
 
-                if (!msgIterator.hasNext() && vertex.isHalted())
+                if (!msgIterator.hasNext() && vertex.isHalted()) {
                     return;
+                }
+                if (vertex.isHalted()) {
+                    vertex.activate();
+                }
 
                 try {
                     vertex.compute(msgIterator);
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AccumulatingAggregatorFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AccumulatingAggregatorFactory.java
index 8f63b6e..851a83a 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AccumulatingAggregatorFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AccumulatingAggregatorFactory.java
@@ -40,7 +40,7 @@
 
     @SuppressWarnings("unchecked")
     @Override
-    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
+    public IAggregatorDescriptor createAggregator(final IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
             RecordDescriptor outRecordDescriptor, int[] aggKeys, int[] partialKeys) throws HyracksDataException {
 
         return new IAggregatorDescriptor() {
@@ -113,7 +113,7 @@
                 for (int i = 0; i < agg.length; i++) {
                     aggOutput[i] = new ArrayBackedValueStorage();
                     try {
-                        agg[i] = aggFactories[i].createAggregateFunction(aggOutput[i]);
+                        agg[i] = aggFactories[i].createAggregateFunction(ctx, aggOutput[i]);
                     } catch (Exception e) {
                         throw new IllegalStateException(e);
                     }
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunction.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunction.java
index 1813dcc..3cf46a2 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunction.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunction.java
@@ -26,6 +26,7 @@
 import org.apache.hadoop.io.WritableComparable;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
@@ -53,9 +54,9 @@
     private MsgList msgList = new MsgList();
     private boolean keyRead = false;
 
-    public AggregationFunction(IConfigurationFactory confFactory, DataOutput output, boolean isFinalStage,
-            boolean partialAggAsInput) throws HyracksDataException {
-        this.conf = confFactory.createConfiguration();
+    public AggregationFunction(IHyracksTaskContext ctx, IConfigurationFactory confFactory, DataOutput output,
+            boolean isFinalStage, boolean partialAggAsInput) throws HyracksDataException {
+        this.conf = confFactory.createConfiguration(ctx);
         this.output = output;
         this.isFinalStage = isFinalStage;
         this.partialAggAsInput = partialAggAsInput;
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunctionFactory.java
index a09f688..7ce9e1d 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunctionFactory.java
@@ -17,6 +17,7 @@
 
 import java.io.DataOutput;
 
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
@@ -36,8 +37,9 @@
     }
 
     @Override
-    public IAggregateFunction createAggregateFunction(IDataOutputProvider provider) throws HyracksException {
+    public IAggregateFunction createAggregateFunction(IHyracksTaskContext ctx, IDataOutputProvider provider)
+            throws HyracksException {
         DataOutput output = provider.getDataOutput();
-        return new AggregationFunction(confFactory, output, isFinalStage, partialAggAsInput);
+        return new AggregationFunction(ctx, confFactory, output, isFinalStage, partialAggAsInput);
     }
 }
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DatatypeHelper.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/DatatypeHelper.java
similarity index 98%
rename from pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DatatypeHelper.java
rename to pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/DatatypeHelper.java
index ee319c6..f9085c4 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DatatypeHelper.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/DatatypeHelper.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.core.util;
+package edu.uci.ics.pregelix.runtime.touchpoint;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -80,6 +80,7 @@
                 throw new HyracksDataException(e);
             }
         }
+
     }
 
     @SuppressWarnings({ "rawtypes", "unchecked" })
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/PreSuperStepRuntimeHookFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/PreSuperStepRuntimeHookFactory.java
index 5f0ed9e..850ae1e 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/PreSuperStepRuntimeHookFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/PreSuperStepRuntimeHookFactory.java
@@ -39,7 +39,7 @@
 
             @Override
             public void configure(IHyracksTaskContext ctx) throws HyracksDataException {
-                Configuration conf = confFactory.createConfiguration();
+                Configuration conf = confFactory.createConfiguration(ctx);
                 IterationUtils.setProperties(giraphJobId, ctx, conf);
             }
 
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java
index c025f85..05b1542 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java
@@ -44,9 +44,10 @@
 
             @Override
             public void configure(IHyracksTaskContext ctx) throws HyracksDataException {
-                Configuration conf = confFactory.createConfiguration();
+                Configuration conf = confFactory.createConfiguration(ctx);
                 try {
                     TaskAttemptContext mapperContext = ctxFactory.createContext(conf, new TaskAttemptID());
+                    mapperContext.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
                     Vertex.setContext(mapperContext);
                     BspUtils.setDefaultConfiguration(conf);
                 } catch (Exception e) {
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdPartitionComputerFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdPartitionComputerFactory.java
index 5eff497..5b4b1f0 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdPartitionComputerFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdPartitionComputerFactory.java
@@ -24,20 +24,22 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import edu.uci.ics.pregelix.dataflow.std.base.ISerializerDeserializerFactory;
 
 public class VertexIdPartitionComputerFactory<K extends Writable, V extends Writable> implements
         ITuplePartitionComputerFactory {
     private static final long serialVersionUID = 1L;
-    private final ISerializerDeserializer<K> keyIO;
+    private final ISerializerDeserializerFactory<K> keyIOFactory;
 
-    public VertexIdPartitionComputerFactory(ISerializerDeserializer<K> keyIO) {
-        this.keyIO = keyIO;
+    public VertexIdPartitionComputerFactory(ISerializerDeserializerFactory<K> keyIOFactory) {
+        this.keyIOFactory = keyIOFactory;
     }
 
     public ITuplePartitionComputer createPartitioner() {
         return new ITuplePartitionComputer() {
             private final ByteBufferInputStream bbis = new ByteBufferInputStream();
             private final DataInputStream dis = new DataInputStream(bbis);
+            private final ISerializerDeserializer<K> keyIO = keyIOFactory.getSerializerDeserializer();
 
             public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
                 int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/WritableSerializerDeserializerFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/WritableSerializerDeserializerFactory.java
new file mode 100644
index 0000000..96f781c
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/WritableSerializerDeserializerFactory.java
@@ -0,0 +1,21 @@
+package edu.uci.ics.pregelix.runtime.touchpoint;
+
+import org.apache.hadoop.io.Writable;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.pregelix.dataflow.std.base.ISerializerDeserializerFactory;
+
+public class WritableSerializerDeserializerFactory<T extends Writable> implements ISerializerDeserializerFactory<T> {
+    private static final long serialVersionUID = 1L;
+    private final Class<T> clazz;
+
+    public WritableSerializerDeserializerFactory(Class<T> clazz) {
+        this.clazz = clazz;
+    }
+
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    @Override
+    public ISerializerDeserializer getSerializerDeserializer() {
+        return DatatypeHelper.createSerializerDeserializer(clazz);
+    }
+}