reintegrate fullstack_dynamic_deployment
diff --git a/.gitignore b/.gitignore
index 328b7d3..31c9fc1 100644
--- a/.gitignore
+++ b/.gitignore
@@ -3,3 +3,14 @@
 .settings
 .project
 ClusterControllerService
+actual
+build
+edu.uci.ics.hyracks.control.nc.NodeControllerService
+exception
+expected
+teststore1
+teststore2
+derby.log
+hadoop-conf-tmp
+metastore_db
+teststore
diff --git a/hivesterix/hivesterix-dist/src/main/resources/conf/cluster.properties b/hivesterix/hivesterix-dist/src/main/resources/conf/cluster.properties
index 2d2401a..056cce4 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/conf/cluster.properties
+++ b/hivesterix/hivesterix-dist/src/main/resources/conf/cluster.properties
@@ -4,9 +4,6 @@
 #The CC port for Hyracks cluster management
 CC_CLUSTERPORT=1099
 
-#The directory of hyracks binaries
-HYRACKS_HOME=../../../../hyracks
-
 #The tmp directory for cc to install jars
 CCTMP_DIR=/tmp/t1
 
@@ -29,9 +26,11 @@
 FRAME_SIZE=65536
 
 #CC JAVA_OPTS
-CCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7001,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+CCJAVA_OPTS="-Xmx1g -Djava.util.logging.config.file=logging.properties"
+# debug option: CCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7001,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
 # Yourkit option: -agentpath:/grid/0/dev/vborkar/tools/yjp-10.0.4/bin/linux-x86-64/libyjpagent.so=port=20001"
 
 #NC JAVA_OPTS
-NCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7002,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
-
+NCJAVA_OPTS="-Xmx1g -Djava.util.logging.config.file=logging.properties"
+# debug option: NCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7002,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+# Yourkit option: -agentpath:/grid/0/dev/vborkar/tools/yjp-10.0.4/bin/linux-x86-64/libyjpagent.so=port=20001"
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh
index 8c9ae76..1b44d09 100755
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/getip.sh
@@ -1,3 +1,21 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 #get the OS
 OS_NAME=`uname -a|awk '{print $1}'`
 LINUX_OS='Linux'
@@ -6,10 +24,10 @@
 then
         #Get IP Address
         IPADDR=`/sbin/ifconfig eth0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
-	if [ "$IPADDR" = "" ]
+    	if [ "$IPADDR" = "" ]
         then
-                IPADDR=`/sbin/ifconfig em1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
-        fi
+		IPADDR=`/sbin/ifconfig em1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+        fi 
 	if [ "$IPADDR" = "" ]
         then
 		IPADDR=`/sbin/ifconfig lo | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/hive b/hivesterix/hivesterix-dist/src/main/resources/scripts/hive
index f98f340..38a9e33 100755
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/hive
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/hive
@@ -81,7 +81,7 @@
   exit 3;
 fi
 
-CLASSPATH=${CLASSPATH}:${HIVE_LIB}/a-hive-path.jar
+CLASSPATH=${CLASSPATH}:${HIVE_LIB}/a-hive-patch.jar
 
 for f in ${HIVE_LIB}/*.jar; do
   CLASSPATH=${CLASSPATH}:$f;
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/hivesterixcc b/hivesterix/hivesterix-dist/src/main/resources/scripts/hivesterixcc
new file mode 100755
index 0000000..d2adffd
--- /dev/null
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/hivesterixcc
@@ -0,0 +1,122 @@
+#!/bin/sh
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
+# 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
+
+#add the desired hive patch
+CLASSPATH=${CLASSPATH}:${REPO}/a-hive-patch.jar
+
+#add the desired jetty jar
+for f in ${REPO}/servlet-api-3*.jar; do
+  CLASSPATH=${CLASSPATH}:$f;
+done
+
+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="hivesterixcc" \
+  -Dapp.pid="$$" \
+  -Dapp.repo="$REPO" \
+  -Dapp.home="$BASEDIR" \
+  -Dbasedir="$BASEDIR" \
+  edu.uci.ics.hyracks.control.cc.CCDriver \
+  "$@"
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/hivesterixnc b/hivesterix/hivesterix-dist/src/main/resources/scripts/hivesterixnc
new file mode 100755
index 0000000..40f3ec6
--- /dev/null
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/hivesterixnc
@@ -0,0 +1,117 @@
+#!/bin/sh
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
+# 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
+
+#add the desired hive patch
+CLASSPATH=${CLASSPATH}:${REPO}/a-hive-patch.jar
+
+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="hivesterixnc" \
+  -Dapp.pid="$$" \
+  -Dapp.repo="$REPO" \
+  -Dapp.home="$BASEDIR" \
+  -Dbasedir="$BASEDIR" \
+  edu.uci.ics.hyracks.control.nc.NCDriver \
+  "$@"
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/startAllNCs.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/startAllNCs.sh
index d30da26..34a318d 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/startAllNCs.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/startAllNCs.sh
@@ -1,6 +1,24 @@
-PREGELIX_PATH=`pwd`
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
+HIVESTERIX_PATH=`pwd`
 
 for i in `cat conf/slaves`
 do
-   ssh $i "cd ${PREGELIX_PATH}; export JAVA_HOME=${JAVA_HOME}; bin/startnc.sh"
+   ssh $i "cd ${HIVESTERIX_PATH}; export JAVA_HOME=${JAVA_HOME}; bin/startnc.sh"
 done
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/startCluster.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/startCluster.sh
index 6aa9161..cddf9a4 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/startCluster.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/startCluster.sh
@@ -1,19 +1,21 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 bin/startcc.sh
 sleep 5
 bin/startAllNCs.sh
-
-. conf/cluster.properties
-# do we need to specify the version somewhere?
-hyrackcmd=`ls ${HYRACKS_HOME}/hyracks-cli/target/hyracks-cli-*-binary-assembly/bin/hyrackscli`
-# find zip file
-appzip=`ls $PWD/../hivesterix-dist-*-binary-assembly.zip`
-
-[ -f $hyrackcmd ] || { echo "Hyracks commandline is missing"; exit -1;}
-[ -f $appzip ] || { echo "Genomix binary-assembly.zip is missing"; exit -1;}
-
-CCHOST_NAME=`cat conf/master`
-
-IPADDR=`bin/getip.sh`
-echo "connect to \"${IPADDR}:${CC_CLIENTPORT}\"; create application hivesterix \"$appzip\";" | $hyrackcmd 
-echo ""
-
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/startDebugNc.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/startDebugNc.sh
index fe6cf27..cab8bb3 100755
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/startDebugNc.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/startDebugNc.sh
@@ -1,3 +1,21 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 hostname
 
 #Get the IP address of the cc
@@ -40,11 +58,11 @@
 #Set JAVA_OPTS
 export JAVA_OPTS=$NCJAVA_OPTS2
 
-cd $HYRACKS_HOME
-HYRACKS_HOME=`pwd`
+HIVESTERIX_HOME=`pwd`
 
 #Enter the temp dir
 cd $NCTMP_DIR2
 
 #Launch hyracks nc
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS2}" &> $NCLOGS_DIR2/$NODEID.log &
+#echo ${HIVESTERIX_HOME}/bin/hivesterixnc -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}"
+${HIVESTERIX_HOME}/bin/hivesterixnc -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_DIRS2}" &> $NCLOGS_DIR2/$NODEID.log &
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/startcc.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/startcc.sh
index 484ecac..213b9eb 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/startcc.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/startcc.sh
@@ -1,4 +1,21 @@
 #!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 hostname
 
 #Import cluster properties
@@ -20,12 +37,15 @@
 export JAVA_HOME=$JAVA_HOME
 export JAVA_OPTS=$CCJAVA_OPTS
 
-#Launch hyracks cc script
-chmod -R 755 $HYRACKS_HOME
+HIVESTERIX_HOME=`pwd`
+
+#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 &
+${HIVESTERIX_HOME}/bin/hivesterixcc -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 &
+${HIVESTERIX_HOME}/bin/hivesterixcc -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/hivesterix/hivesterix-dist/src/main/resources/scripts/startnc.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/startnc.sh
index 23a4c36..904858d 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/startnc.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/startnc.sh
@@ -1,3 +1,21 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 hostname
 
 MY_NAME=`hostname`
@@ -39,11 +57,11 @@
 #Set JAVA_OPTS
 export JAVA_OPTS=$NCJAVA_OPTS
 
-cd $HYRACKS_HOME
-HYRACKS_HOME=`pwd`
+HIVESTERIX_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 &
+#echo ${HIVESTERIX_HOME}/bin/hivesterixnc -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}"
+${HIVESTERIX_HOME}/bin/hivesterixnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -result-ip-address $IPADDR  -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/stopAllNCs.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/stopAllNCs.sh
index 12367c1..456ff68 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/stopAllNCs.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/stopAllNCs.sh
@@ -1,6 +1,24 @@
-PREGELIX_PATH=`pwd`
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
+HIVESTERIX_PATH=`pwd`
 
 for i in `cat conf/slaves`
 do
-   ssh $i "cd ${PREGELIX_PATH}; bin/stopnc.sh"
+   ssh $i "cd ${HIVESTERIX_PATH}; bin/stopnc.sh"
 done
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/stopCluster.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/stopCluster.sh
index 4889934..6b829aa 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/stopCluster.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/stopCluster.sh
@@ -1,3 +1,21 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 bin/stopAllNCs.sh
 sleep 2
 bin/stopcc.sh
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/stopcc.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/stopcc.sh
index c2f525a..d7f78a8 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/stopcc.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/stopcc.sh
@@ -1,8 +1,37 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 hostname
 . 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=hivesterixcc'|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=hivesterixcc'|awk '{print $2}'`
+fi
+
 echo $PID
 kill -9 $PID
 
diff --git a/hivesterix/hivesterix-dist/src/main/resources/scripts/stopnc.sh b/hivesterix/hivesterix-dist/src/main/resources/scripts/stopnc.sh
index 35c4794..5dc4ccd 100644
--- a/hivesterix/hivesterix-dist/src/main/resources/scripts/stopnc.sh
+++ b/hivesterix/hivesterix-dist/src/main/resources/scripts/stopnc.sh
@@ -1,8 +1,26 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 hostname
 . 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=hivesterixnc'|awk '{print $2}'`
 
 if [ "$PID" == "" ]; then
   PID=`ps -ef|grep ${USER}|grep java|grep 'hyracks'|awk '{print $2}'`
@@ -10,7 +28,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=hivesterixnc'|awk '{print $2}'`
 fi
 
 echo $PID
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 1b079d6..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,6 +16,7 @@
 
 import java.io.Serializable;
 
+import edu.uci.ics.hyracks.api.job.IJobSerializerDeserializerContainer;
 import edu.uci.ics.hyracks.api.messages.IMessageBroker;
 
 /**
@@ -35,4 +36,7 @@
     public void setMessageBroker(IMessageBroker messageBroker);
 
     public IMessageBroker getMessageBroker();
+
+    public IJobSerializerDeserializerContainer getJobSerializerDeserializerContainer();
+
 }
\ 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 88df49f..0467eae 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;
 
@@ -34,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 {
@@ -76,10 +81,18 @@
 
         private final byte[] acggfBytes;
         private final EnumSet<JobFlag> jobFlags;
+        private final DeploymentId deploymentId;
 
         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
@@ -94,6 +107,10 @@
         public EnumSet<JobFlag> getJobFlags() {
             return jobFlags;
         }
+
+        public DeploymentId getDeploymentId() {
+            return deploymentId;
+        }
     }
 
     public static class GetDatasetDirectoryServiceInfoFunction extends Function {
@@ -200,4 +217,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 033fc02..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;
@@ -56,6 +59,13 @@
     }
 
     @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);
+    }
+
+    @Override
     public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception {
         HyracksClientInterfaceFunctions.GetDatasetDirectoryServiceInfoFunction gddsf = new HyracksClientInterfaceFunctions.GetDatasetDirectoryServiceInfoFunction();
         return (NetworkAddress) rpci.call(ipcHandle, gddsf);
@@ -79,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 8274416..6af8dd9 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
@@ -14,12 +14,25 @@
  */
 package edu.uci.ics.hyracks.api.client;
 
+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;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.entity.FileEntity;
+import org.apache.http.impl.client.DefaultHttpClient;
 
 import edu.uci.ics.hyracks.api.client.impl.JobSpecificationActivityClusterGraphGeneratorFactory;
 import edu.uci.ics.hyracks.api.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;
 import edu.uci.ics.hyracks.api.job.JobId;
@@ -107,4 +120,60 @@
     public ClusterTopology getClusterTopology() throws Exception {
         return hci.getClusterTopology();
     }
+
+    @Override
+    public DeploymentId deployBinary(List<String> jars) throws Exception {
+        /** generate a deployment id */
+        DeploymentId deploymentId = new DeploymentId(UUID.randomUUID().toString());
+        List<URL> binaryURLs = new ArrayList<URL>();
+        if (jars != null && jars.size() > 0) {
+            HttpClient hc = new DefaultHttpClient();
+            /** upload jars through a http client one-by-one to the CC server */
+            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) {
+                    hci.unDeployBinary(deploymentId);
+                    throw new HyracksException(response.getStatusLine().toString());
+                }
+                /** add the uploaded URL address into the URLs of jars to be deployed at NCs */
+                binaryURLs.add(new URL(url));
+            }
+        }
+        /**deploy the URLs to the CC and NCs*/
+        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 fe4094f..41b07d7 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
@@ -15,9 +15,11 @@
 package edu.uci.ics.hyracks.api.client;
 
 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;
@@ -110,4 +112,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 6fdf638..aabc351 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;
@@ -37,4 +40,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/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/deployment/DeploymentId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/deployment/DeploymentId.java
new file mode 100644
index 0000000..8d8d728
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/deployment/DeploymentId.java
@@ -0,0 +1,54 @@
+/*
+ * 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.deployment;
+
+import java.io.Serializable;
+
+/**
+ * The representation of a deployment id
+ * 
+ * @author yingyib
+ *
+ */
+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/IJobSerializerDeserializer.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializer.java
new file mode 100644
index 0000000..a105953
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializer.java
@@ -0,0 +1,71 @@
+/*
+ * 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;
+
+/**
+ * The serializer/deserializer/classloader interface for job/task information such as job specifications, activity graphs and so on.
+ * 
+ * @author yingyib
+ */
+public interface IJobSerializerDeserializer {
+
+    /**
+     * Deserialize the bytes to an object
+     * 
+     * @param bytes
+     *            the binary content of an object
+     * @return the deserialized object
+     * @throws HyracksException
+     */
+    public Object deserialize(byte[] bytes) throws HyracksException;
+
+    /**
+     * Serialize a object into bytes
+     * 
+     * @param object
+     *            a Serializable Java object
+     * @return
+     *         the byte array which contains the binary content of the input object
+     * @throws HyracksException
+     */
+    public byte[] serialize(Serializable object) throws HyracksException;
+
+    /**
+     * Load a class by its name
+     * 
+     * @param className
+     *            the name of the class
+     * @return
+     * @throws HyracksException
+     */
+    public Class<?> loadClass(String className) throws HyracksException;
+
+    /**
+     * 
+     * @param binaryURLs
+     * @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 764348f..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,19 @@
         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;
@@ -55,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-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 506a870..9b8a996 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
@@ -40,6 +40,7 @@
 import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
 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;
@@ -49,6 +50,8 @@
 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.ApplicationMessageWork;
+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;
@@ -58,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;
@@ -73,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;
@@ -126,6 +131,8 @@
 
     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");
@@ -172,6 +179,8 @@
         sweeper = new DeadNodeSweeper();
         datasetDirectoryService = new DatasetDirectoryService(ccConfig.jobHistorySize);
         jobCounter = 0;
+
+        deploymentRunMap = new HashMap<DeploymentId, DeploymentRun>();
     }
 
     private static ClusterTopology computeClusterTopology(CCConfig ccConfig) throws Exception {
@@ -326,8 +335,8 @@
                 case START_JOB: {
                     HyracksClientInterfaceFunctions.StartJobFunction sjf = (HyracksClientInterfaceFunctions.StartJobFunction) fn;
                     JobId jobId = createJobId();
-                    workQueue.schedule(new JobStartWork(ClusterControllerService.this, sjf.getACGGFBytes(), sjf
-                            .getJobFlags(), jobId, new IPCResponder<JobId>(handle, mid)));
+                    workQueue.schedule(new JobStartWork(ClusterControllerService.this, sjf.getDeploymentId(), sjf
+                            .getACGGFBytes(), sjf.getJobFlags(), jobId, new IPCResponder<JobId>(handle, mid)));
                     return;
                 }
 
@@ -373,6 +382,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()));
@@ -414,6 +437,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()));
@@ -472,7 +502,7 @@
                 case SEND_APPLICATION_MESSAGE: {
                     CCNCFunctions.SendApplicationMessageFunction rsf = (CCNCFunctions.SendApplicationMessageFunction) fn;
                     workQueue.schedule(new ApplicationMessageWork(ClusterControllerService.this, rsf.getMessage(), rsf
-                            .getNodeId()));
+                            .getDeploymentId(), rsf.getNodeId()));
                     return;
                 }
 
@@ -496,4 +526,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/job/JobRun.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
index 33e1ff6..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,6 +46,8 @@
 import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
 
 public class JobRun implements IJobStatusConditionVariable {
+    private final DeploymentId deploymentId;
+
     private final JobId jobId;
 
     private final IActivityClusterGraphGenerator acgg;
@@ -81,8 +84,9 @@
 
     private Exception pendingException;
 
-    public JobRun(ClusterControllerService ccs, JobId jobId, IActivityClusterGraphGenerator acgg,
-            EnumSet<JobFlag> jobFlags) {
+    public JobRun(ClusterControllerService ccs, DeploymentId deploymentId, JobId jobId,
+            IActivityClusterGraphGenerator acgg, EnumSet<JobFlag> jobFlags) {
+        this.deploymentId = deploymentId;
         this.jobId = jobId;
         this.acgg = acgg;
         this.acg = acgg.initialize();
@@ -96,6 +100,10 @@
         connectorPolicyMap = new HashMap<ConnectorDescriptorId, IConnectorPolicy>();
     }
 
+    public DeploymentId getDeploymentId() {
+        return deploymentId;
+    }
+
     public JobId getJobId() {
         return jobId;
     }
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 34b7dc7..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,6 +428,7 @@
     }
 
     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 Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = new HashMap<ConnectorDescriptorId, IConnectorPolicy>(
@@ -443,8 +445,8 @@
                 }
                 try {
                     byte[] jagBytes = changed ? JavaSerializationUtils.serialize(acg) : null;
-                    node.getNodeController().startTasks(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
new file mode 100755
index 0000000..35165de
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
@@ -0,0 +1,121 @@
+/*
+ * 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.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;
+
+import javax.servlet.ServletException;
+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.work.SynchronizableWork;
+
+public class ApplicationInstallationHandler extends AbstractHandler {
+    private ClusterControllerService ccs;
+
+    public ApplicationInstallationHandler(ClusterControllerService ccs) {
+        this.ccs = ccs;
+    }
+
+    @Override
+    public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response)
+            throws IOException, ServletException {
+        try {
+            while (target.startsWith("/")) {
+                target = target.substring(1);
+            }
+            while (target.endsWith("/")) {
+                target = target.substring(0, target.length() - 1);
+            }
+            String[] parts = target.split("/");
+            if (parts.length != 1) {
+                return;
+            }
+            final String[] 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 {
+                        FileUtils.forceMkdir(new File(deploymentDir));
+                        String fileName = params[1];
+                        File jarFile = new File(deploymentDir, fileName);
+                        os = new FileOutputStream(jarFile);
+                    }
+                }
+                OutputStreamGetter r = new OutputStreamGetter();
+                try {
+                    ccs.getWorkQueue().scheduleAndSync(r);
+                } catch (Exception e) {
+                    throw new IOException(e);
+                }
+                try {
+                    IOUtils.copyLarge(request.getInputStream(), r.os);
+                } finally {
+                    r.os.close();
+                }
+            } else if (HttpMethods.GET.equals(request.getMethod())) {
+                class InputStreamGetter extends SynchronizableWork {
+                    private InputStream is;
+
+                    @Override
+                    protected void doRun() throws Exception {
+                        String fileName = params[1];
+                        File jarFile = new File(deploymentDir, fileName);
+                        is = new FileInputStream(jarFile);
+                    }
+                }
+                InputStreamGetter r = new InputStreamGetter();
+                try {
+                    ccs.getWorkQueue().scheduleAndSync(r);
+                } catch (Exception e) {
+                    throw new IOException(e);
+                }
+                if (r.is == null) {
+                    response.setStatus(HttpServletResponse.SC_NOT_FOUND);
+                } else {
+                    response.setContentType("application/octet-stream");
+                    response.setStatus(HttpServletResponse.SC_OK);
+                    try {
+                        IOUtils.copyLarge(r.is, response.getOutputStream());
+                    } finally {
+                        r.is.close();
+                    }
+                }
+            }
+            baseRequest.setHandled(true);
+        } catch (IOException e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
index e39e766..cd7a684 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
@@ -70,6 +70,11 @@
 
         addHandler(createAdminConsoleHandler());
         addHandler(createStaticResourcesHandler());
+
+        /** the service of uploading/downloading deployment jars */
+        handler = new ContextHandler("/applications");
+        handler.setHandler(new ApplicationInstallationHandler(ccs));
+        addHandler(handler);
     }
 
     private Handler createAdminConsoleHandler() {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java
index b880c8a..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,14 +14,14 @@
  */
 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.api.util.JavaSerializationUtils;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.deployment.DeploymentUtils;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
 /**
@@ -31,11 +31,13 @@
 
     private static final Logger LOGGER = Logger.getLogger(ApplicationMessageWork.class.getName());
     private byte[] message;
+    private DeploymentId deploymentId;
     private String nodeId;
     private ClusterControllerService ccs;
 
-    public ApplicationMessageWork(ClusterControllerService ccs, byte[] message, String nodeId) {
+    public ApplicationMessageWork(ClusterControllerService ccs, byte[] message, DeploymentId deploymentId, String nodeId) {
         this.ccs = ccs;
+        this.deploymentId = deploymentId;
         this.nodeId = nodeId;
         this.message = message;
     }
@@ -44,17 +46,16 @@
     public void run() {
         final ICCApplicationContext ctx = ccs.getApplicationContext();
         try {
-            final IMessage data = (IMessage) JavaSerializationUtils.deserialize(message);
+            final IMessage data = (IMessage) DeploymentUtils.deserialize(message, deploymentId, ctx);
             ccs.getExecutor().execute(new Runnable() {
                 @Override
                 public void run() {
                     ctx.getMessageBroker().receivedMessage(data, nodeId);
                 }
             });
-        } 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);
         }
     }
 
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..1a889f8
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/CliDeployBinaryWork.java
@@ -0,0 +1,108 @@
+/*
+ * 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;
+
+/***
+ * This is the work happens on the CC for a dynamic deployment.
+ * It first deploys the jar to CC application context.
+ * Then, it remotely calls each NC service to deploy the jars listed as http URLs.
+ * NOTE: in current implementation, a user cannot deploy with the same deployment id simultaneously.
+ * 
+ * @author yingyib
+ */
+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);
+
+            /** The following call prevents a user to deploy with the same deployment id simultaneously. */
+            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..d54444c
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/CliUnDeployBinaryWork.java
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/***
+ * This is the work happens on the CC for an undeployment.
+ * It first undeploys the jar to CC application context by giving it a deployment id.
+ * Then, it remotely calls each NC service to undeploy by giving them a deployment id.
+ * NOTE: in current implementation, a user cannot undeploy with the same deployment id simultaneously.
+ * 
+ * @author yingyib
+ */
+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);
+            
+            /** The following call prevents a user to undeploy with the same deployment id simultaneously. */
+            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/JobStartWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
index f7d6c4c..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,15 +16,16 @@
 
 import java.util.EnumSet;
 
+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;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
+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;
 
@@ -32,11 +33,13 @@
     private final ClusterControllerService ccs;
     private final byte[] acggfBytes;
     private final EnumSet<JobFlag> jobFlags;
+    private final DeploymentId deploymentId;
     private final JobId jobId;
     private final IResultCallback<JobId> callback;
 
-    public JobStartWork(ClusterControllerService ccs, 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;
@@ -48,10 +51,10 @@
     protected void doRun() throws Exception {
         try {
             final CCApplicationContext appCtx = ccs.getApplicationContext();
-            IActivityClusterGraphGeneratorFactory acggf = (IActivityClusterGraphGeneratorFactory) JavaSerializationUtils
-                    .deserialize(acggfBytes);
+            IActivityClusterGraphGeneratorFactory acggf = (IActivityClusterGraphGeneratorFactory) DeploymentUtils
+                    .deserialize(acggfBytes, deploymentId, appCtx);
             IActivityClusterGraphGenerator acgg = acggf.createActivityClusterGraphGenerator(jobId, appCtx, jobFlags);
-            JobRun run = new JobRun(ccs, jobId, acgg, 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/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..c4c8873
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NotifyDeployBinaryWork.java
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+/***
+ * This is the work happens on the CC when CC gets a deployment or undeployment notification status message from one NC.
+ * 
+ * @author yingyib
+ */
+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() {
+        /** triggered remotely by a NC to notify that the NC is deployed */
+        DeploymentRun dRun = ccs.getDeploymentRun(deploymentId);
+        dRun.notifyDeploymentStatus(nodeId, deploymentStatus);
+    }
+
+}
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 5251584..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
@@ -18,6 +18,8 @@
 import java.io.Serializable;
 
 import edu.uci.ics.hyracks.api.application.IApplicationContext;
+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.control.common.context.ServerContext;
 
@@ -25,6 +27,7 @@
     protected ServerContext serverCtx;
     protected Serializable distributedState;
     protected IMessageBroker messageBroker;
+    protected IJobSerializerDeserializerContainer jobSerDeContainer = new JobSerializerDeserializerContainer();
 
     public ApplicationContext(ServerContext serverCtx) throws IOException {
         this.serverCtx = serverCtx;
@@ -44,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 627dd55..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,8 +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.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;
@@ -39,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;
@@ -47,7 +51,7 @@
 
     public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception;
 
-    public void sendApplicationMessageToCC(byte[] data, String nodeId) 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;
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 c589c97..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,7 +31,8 @@
 import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
 
 public interface INodeController {
-    public void startTasks(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;
@@ -37,4 +40,8 @@
     public void cleanUpJoblet(JobId jobId, JobStatus status) 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/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..3a35c25
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/ClassLoaderJobSerializerDeserializer.java
@@ -0,0 +1,102 @@
+/*
+ * 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.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;
+
+/**
+ * This is the IJobSerializerDeserializer implementation for jobs with dynamic deployed jars.
+ * 
+ * @author yingyib
+ *
+ */
+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..84cfbb1
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentRun.java
@@ -0,0 +1,62 @@
+/*
+ * 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.common.deployment;
+
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The class maintain the status of a deployment process and the states
+ * of all slave machines involved in the deployment.
+ * 
+ * @author yingyib
+ */
+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);
+    }
+
+    /**
+     * One notify the deployment status
+     * 
+     * @param nodeId
+     * @param status
+     */
+    public synchronized void notifyDeploymentStatus(String nodeId, DeploymentStatus status) {
+        if (status == DeploymentStatus.SUCCEED) {
+            deploymentNodeIds.remove(nodeId);
+            if (deploymentNodeIds.size() == 0) {
+                deploymentStatus = DeploymentStatus.SUCCEED;
+                notifyAll();
+            }
+        } else {
+            deploymentNodeIds.clear();
+            deploymentStatus = DeploymentStatus.FAIL;
+            notifyAll();
+        }
+    }
+
+    @Override
+    public synchronized DeploymentStatus waitForCompletion() throws Exception {
+        wait();
+        return deploymentStatus;
+    }
+
+}
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentStatus.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentStatus.java
new file mode 100644
index 0000000..4ba50b7
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentStatus.java
@@ -0,0 +1,21 @@
+/*
+ * 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.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..0677e2e
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentUtils.java
@@ -0,0 +1,216 @@
+/*
+ * 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.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;
+
+/**
+ * A utility class which is in charge of the actual work of deployments.
+ * 
+ * @author yingyib
+ */
+public class DeploymentUtils {
+
+    private static final String DEPLOYMENT = "applications";
+
+    /**
+     * undeploy an existing deployment
+     * 
+     * @param deploymentId
+     *            the deployment id
+     * @param container
+     * @param ctx
+     * @throws HyracksException
+     */
+    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);
+        }
+    }
+
+    /**
+     * Deploying jars in NC or CC
+     * 
+     * @param deploymentId
+     *            the deployment id
+     * @param urls
+     *            the jar URLs
+     * @param container
+     *            the container of serailizer/deserializer
+     * @param ctx
+     *            the ServerContext
+     * @param isNC
+     *            true is NC/false is CC
+     * @throws HyracksException
+     */
+    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));
+    }
+
+    /**
+     * Deserialize bytes to an object according to a specific deployment
+     * 
+     * @param bytes
+     *            the bytes to be deserialized
+     * @param deploymentId
+     *            the deployment id
+     * @param appCtx
+     * @return the deserialized object
+     * @throws HyracksException
+     */
+    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);
+        }
+    }
+
+    /**
+     * Load a class from its class name
+     * 
+     * @param className
+     * @param deploymentId
+     * @param appCtx
+     * @return the loaded class
+     * @throws HyracksException
+     */
+    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);
+        }
+    }
+
+    /**
+     * Get the classloader of a specific deployment
+     * 
+     * @param deploymentId
+     * @param appCtx
+     * @return
+     * @throws HyracksException
+     */
+    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);
+        }
+    }
+
+    /**
+     * Download remote Http URLs and return the stored local file URLs
+     * 
+     * @param urls
+     *            the remote Http URLs
+     * @param deploymentDir
+     *            the deployment jar storage directory
+     * @param isNC
+     *            true is NC/false is CC
+     * @return a list of local file URLs
+     * @throws HyracksException
+     */
+    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-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/IDeploymentStatusConditionVariable.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/IDeploymentStatusConditionVariable.java
new file mode 100644
index 0000000..771baaa
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/IDeploymentStatusConditionVariable.java
@@ -0,0 +1,32 @@
+/*
+ * 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.common.deployment;
+
+/**
+ * The conditional variable interface for deployment
+ * 
+ * @author yingyib
+ */
+public interface IDeploymentStatusConditionVariable {
+
+    /**
+     * synchronously wait the deployment to complete
+     * 
+     * @return the deployment status
+     * @throws Exception
+     */
+    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 f6ab9ba..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,12 +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.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;
@@ -81,14 +84,23 @@
         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;
 
+        public DeploymentId getDeploymentId() {
+            return deploymentId;
+        }
+
         public String getNodeId() {
             return nodeId;
         }
@@ -101,8 +113,9 @@
             return serializedMessage;
         }
 
-        public SendApplicationMessageFunction(byte[] data, String nodeId) {
+        public SendApplicationMessageFunction(byte[] data, DeploymentId deploymentId, String nodeId) {
             this.serializedMessage = data;
+            this.deploymentId = deploymentId;
             this.nodeId = nodeId;
         }
 
@@ -581,14 +594,17 @@
     public static class StartTasksFunction extends Function {
         private static final long serialVersionUID = 1L;
 
+        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(JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
+        public StartTasksFunction(DeploymentId deploymentId, JobId jobId, byte[] planBytes,
+                List<TaskAttemptDescriptor> taskDescriptors,
                 Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, EnumSet<JobFlag> flags) {
+            this.deploymentId = deploymentId;
             this.jobId = jobId;
             this.planBytes = planBytes;
             this.taskDescriptors = taskDescriptors;
@@ -601,6 +617,10 @@
             return FunctionId.START_TASKS;
         }
 
+        public DeploymentId getDeploymentId() {
+            return deploymentId;
+        }
+
         public JobId getJobId() {
             return jobId;
         }
@@ -750,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 057a0f4..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,9 +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.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;
@@ -70,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);
@@ -96,8 +105,9 @@
     }
 
     @Override
-    public void sendApplicationMessageToCC(byte[] data, String nodeId) throws Exception {
-        CCNCFunctions.SendApplicationMessageFunction fn = new CCNCFunctions.SendApplicationMessageFunction(data, nodeId);
+    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);
     }
 
@@ -126,4 +136,5 @@
     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 e4355aa..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,10 +40,11 @@
     }
 
     @Override
-    public void startTasks(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(jobId, planBytes, taskDescriptors,
-                connectorPolicies, flags);
+        CCNCFunctions.StartTasksFunction stf = new CCNCFunctions.StartTasksFunction(deploymentId, jobId, planBytes,
+                taskDescriptors, connectorPolicies, flags);
         ipcHandle.send(-1, stf, null);
     }
 
@@ -63,4 +66,16 @@
                 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/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/NodeControllerService.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index e15c60e..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
@@ -47,6 +47,7 @@
 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;
@@ -73,8 +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.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;
@@ -444,13 +447,14 @@
                 case SEND_APPLICATION_MESSAGE: {
                     CCNCFunctions.SendApplicationMessageFunction amf = (CCNCFunctions.SendApplicationMessageFunction) fn;
                     queue.schedule(new ApplicationMessageWork(NodeControllerService.this, amf.getMessage(), amf
-                            .getNodeId()));
+                            .getDeploymentId(), amf.getNodeId()));
                     return;
                 }
                 case START_TASKS: {
                     CCNCFunctions.StartTasksFunction stf = (CCNCFunctions.StartTasksFunction) fn;
-                    queue.schedule(new StartTasksWork(NodeControllerService.this, 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;
                 }
 
@@ -484,14 +488,27 @@
                     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 void sendApplicationMessageToCC(byte[] data, String nodeId) throws Exception {
-        ccs.sendApplicationMessageToCC(data, nodeId);
+    public void sendApplicationMessageToCC(byte[] data, DeploymentId deploymentId, String nodeId) throws Exception {
+        ccs.sendApplicationMessageToCC(data, deploymentId, nodeId);
     }
 
     public IDatasetPartitionManager getDatasetPartitionManager() {
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 d6ea111..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,7 +355,7 @@
     }
 
     @Override
-    public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception {
-        this.ncs.sendApplicationMessageToCC(message, 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/work/ApplicationMessageWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ApplicationMessageWork.java
index a1499b8..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,12 +14,12 @@
  */
 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.api.util.JavaSerializationUtils;
+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;
@@ -30,11 +30,13 @@
 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;
 
-    public ApplicationMessageWork(NodeControllerService ncs, byte[] message, String nodeId) {
+    public ApplicationMessageWork(NodeControllerService ncs, byte[] message, DeploymentId deploymentId, String nodeId) {
         this.ncs = ncs;
+        this.deploymentId = deploymentId;
         this.nodeId = nodeId;
         this.message = message;
     }
@@ -43,16 +45,15 @@
     public void run() {
         NCApplicationContext ctx = ncs.getApplicationContext();
         try {
-            IMessage data = (IMessage) JavaSerializationUtils.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);
         }
     }
 
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..e5f1bc1
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DeployBinaryWork.java
@@ -0,0 +1,65 @@
+/*
+ * 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.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;
+
+/**
+ * Deploy binary work that happens at each NC
+ * 
+ * @author yingyib
+ *
+ */
+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() {
+        DeploymentStatus status;
+        try {
+            DeploymentUtils.deploy(deploymentId, binaryURLs, ncs.getApplicationContext()
+                    .getJobSerializerDeserializerContainer(), ncs.getServerContext(), true);
+            status = DeploymentStatus.SUCCEED;
+        } catch (Exception e) {
+            status = DeploymentStatus.FAIL;
+            e.printStackTrace();
+        }
+        try {
+            IClusterController ccs = ncs.getClusterController();
+            ccs.notifyDeployBinary(deploymentId, ncs.getId(), status);
+        } 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/StartTasksWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
index fffecc2..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,13 +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.api.util.JavaSerializationUtils;
+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;
@@ -60,6 +61,8 @@
 
     private final NodeControllerService ncs;
 
+    private final DeploymentId deploymentId;
+
     private final JobId jobId;
 
     private final byte[] acgBytes;
@@ -70,10 +73,11 @@
 
     private final EnumSet<JobFlag> flags;
 
-    public StartTasksWork(NodeControllerService ncs, 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.deploymentId = deploymentId;
         this.jobId = jobId;
         this.acgBytes = acgBytes;
         this.taskDescriptors = taskDescriptors;
@@ -85,8 +89,8 @@
     public void run() {
         try {
             NCApplicationContext appCtx = ncs.getApplicationContext();
-            final Joblet joblet = getOrCreateLocalJoblet(jobId, appCtx, acgBytes == null ? null
-                    : (ActivityClusterGraph) JavaSerializationUtils.deserialize(acgBytes));
+            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() {
@@ -164,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..605d63f
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/UnDeployBinaryWork.java
@@ -0,0 +1,58 @@
+/*
+ * 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.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;
+
+/**
+ * undeploy binaries regarding to a deployment id
+ * 
+ * @author yingyib
+ */
+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() {
+        DeploymentStatus status;
+        try {
+            DeploymentUtils.undeploy(deploymentId, ncs.getApplicationContext().getJobSerializerDeserializerContainer(),
+                    ncs.getServerContext());
+            status = DeploymentStatus.SUCCEED;
+        } catch (Exception e) {
+            status = DeploymentStatus.FAIL;
+        }
+        try {
+            IClusterController ccs = ncs.getClusterController();
+            ccs.notifyDeployBinary(deploymentId, ncs.getId(), status);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+}
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-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 0c64d7e..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 {
@@ -68,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/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 72256f9..1f071bf 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,90 +139,86 @@
             } while (!terminate);
 
             start = System.currentTimeMillis();
-            runHDFSWRite(jobGen);
-            runCleanup(jobGen);
+            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) {
             throw new HyracksException(e);
         }
     }
 
-    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(job, profiling ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
+        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();
-        // TODO: Fix this step to use Yarn
-        //hcc.createApplication(applicationName, appZip);
-        end = System.currentTimeMillis();
         LOG.info("jar deployment finished " + (end - start) + "ms");
+        return deploymentId;
     }
 }
 
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 77fd1a7..ce1a34d 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.StorageManagerInterface;
 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());
@@ -239,7 +239,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;
@@ -315,7 +315,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);
@@ -382,7 +382,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 fe2fcac..ff2f6a0 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 f1eceb7..ee385f1 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 314c393..40b5f45 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 0c3db38..3351a2c 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/resources/conf/cluster.properties b/pregelix/pregelix-core/src/main/resources/conf/cluster.properties
index 2d2401a..056cce4 100644
--- a/pregelix/pregelix-core/src/main/resources/conf/cluster.properties
+++ b/pregelix/pregelix-core/src/main/resources/conf/cluster.properties
@@ -4,9 +4,6 @@
 #The CC port for Hyracks cluster management
 CC_CLUSTERPORT=1099
 
-#The directory of hyracks binaries
-HYRACKS_HOME=../../../../hyracks
-
 #The tmp directory for cc to install jars
 CCTMP_DIR=/tmp/t1
 
@@ -29,9 +26,11 @@
 FRAME_SIZE=65536
 
 #CC JAVA_OPTS
-CCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7001,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+CCJAVA_OPTS="-Xmx1g -Djava.util.logging.config.file=logging.properties"
+# debug option: CCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7001,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
 # Yourkit option: -agentpath:/grid/0/dev/vborkar/tools/yjp-10.0.4/bin/linux-x86-64/libyjpagent.so=port=20001"
 
 #NC JAVA_OPTS
-NCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7002,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
-
+NCJAVA_OPTS="-Xmx1g -Djava.util.logging.config.file=logging.properties"
+# debug option: NCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7002,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+# Yourkit option: -agentpath:/grid/0/dev/vborkar/tools/yjp-10.0.4/bin/linux-x86-64/libyjpagent.so=port=20001"
diff --git a/pregelix/pregelix-core/src/main/resources/hyracks-deployment.properties b/pregelix/pregelix-core/src/main/resources/hyracks-deployment.properties
deleted file mode 100644
index d5d7cd0..0000000
--- a/pregelix/pregelix-core/src/main/resources/hyracks-deployment.properties
+++ /dev/null
@@ -1 +0,0 @@
-nc.bootstrap.class=edu.uci.ics.pregelix.runtime.bootstrap.NCBootstrapImpl
\ No newline at end of file
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/getip.sh b/pregelix/pregelix-core/src/main/resources/scripts/getip.sh
index a691c0f..1b44d09 100755
--- a/pregelix/pregelix-core/src/main/resources/scripts/getip.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/getip.sh
@@ -1,3 +1,21 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 #get the OS
 OS_NAME=`uname -a|awk '{print $1}'`
 LINUX_OS='Linux'
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/pregelix b/pregelix/pregelix-core/src/main/resources/scripts/pregelix
index b1a2f74..7232ccc 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/pregelix
+++ b/pregelix/pregelix-core/src/main/resources/scripts/pregelix
@@ -1,22 +1,20 @@
 #!/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
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
 #
-#       http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT 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
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..c1ee3f2
--- /dev/null
+++ b/pregelix/pregelix-core/src/main/resources/scripts/pregelixcc
@@ -0,0 +1,114 @@
+#!/bin/sh
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
+# 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..c01b4b4
--- /dev/null
+++ b/pregelix/pregelix-core/src/main/resources/scripts/pregelixnc
@@ -0,0 +1,115 @@
+#!/bin/sh
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
+# 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/startAllNCs.sh b/pregelix/pregelix-core/src/main/resources/scripts/startAllNCs.sh
index d30da26..821be00 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/startAllNCs.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startAllNCs.sh
@@ -1,3 +1,21 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 PREGELIX_PATH=`pwd`
 
 for i in `cat conf/slaves`
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/startCluster.sh b/pregelix/pregelix-core/src/main/resources/scripts/startCluster.sh
index a0c2063..cddf9a4 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/startCluster.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startCluster.sh
@@ -1,3 +1,21 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 bin/startcc.sh
 sleep 5
 bin/startAllNCs.sh
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/startDebugNc.sh b/pregelix/pregelix-core/src/main/resources/scripts/startDebugNc.sh
index fe6cf27..bb43686 100755
--- a/pregelix/pregelix-core/src/main/resources/scripts/startDebugNc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startDebugNc.sh
@@ -1,3 +1,21 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 hostname
 
 #Get the IP address of the cc
@@ -40,11 +58,11 @@
 #Set JAVA_OPTS
 export JAVA_OPTS=$NCJAVA_OPTS2
 
-cd $HYRACKS_HOME
-HYRACKS_HOME=`pwd`
+PREGELIX_HOME=`pwd`
 
 #Enter the temp dir
 cd $NCTMP_DIR2
 
 #Launch hyracks nc
-$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR  -data-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS2}" &> $NCLOGS_DIR2/$NODEID.log &
+#echo ${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}"
+${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_DIRS2}" &> $NCLOGS_DIR2/$NODEID.log &
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh b/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
index 133b604..282a6e7 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
@@ -1,4 +1,21 @@
 #!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 hostname
 
 #Import cluster properties
@@ -20,12 +37,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..970c30b 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startnc.sh
@@ -1,3 +1,21 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 hostname
 
 MY_NAME=`hostname`
@@ -39,11 +57,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/stopAllNCs.sh b/pregelix/pregelix-core/src/main/resources/scripts/stopAllNCs.sh
index 12367c1..dc576d1 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/stopAllNCs.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/stopAllNCs.sh
@@ -1,3 +1,21 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 PREGELIX_PATH=`pwd`
 
 for i in `cat conf/slaves`
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/stopCluster.sh b/pregelix/pregelix-core/src/main/resources/scripts/stopCluster.sh
index 4889934..6b829aa 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/stopCluster.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/stopCluster.sh
@@ -1,3 +1,21 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 bin/stopAllNCs.sh
 sleep 2
 bin/stopcc.sh
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/stopcc.sh b/pregelix/pregelix-core/src/main/resources/scripts/stopcc.sh
index c2f525a..8f94fb7 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/stopcc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/stopcc.sh
@@ -1,8 +1,37 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 hostname
 . 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..26ef089 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/stopnc.sh
@@ -1,8 +1,26 @@
+#!/bin/bash
+#
+#------------------------------------------------------------------------
+# 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.
+# ------------------------------------------------------------------------
+#
+
 hostname
 . 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}'`
@@ -10,7 +28,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-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/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/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..343ba7e 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,12 @@
         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())) {
@@ -109,8 +106,6 @@
                     writer.close();
                 } catch (Exception e) {
                     throw new HyracksDataException(e);
-                } finally {
-                    Thread.currentThread().setContextClassLoader(ctxCL);
                 }
             }
 
@@ -135,10 +130,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 37da84f..7ca7e5d 100644
--- a/pregelix/pregelix-example/pom.xml
+++ b/pregelix/pregelix-example/pom.xml
@@ -69,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>
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/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 4bf83e6..87bc40d 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
@@ -62,7 +62,12 @@
     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 DATA_PATH4 = "data/clique2/clique.txt";
+    private static final String HDFS_PATH4 = "/clique2/";
+
+    private static final String DATA_PATH5 = "data/clique3/clique.txt";
+    private static final String HDFS_PATH5 = "/clique3/";
+
     private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
     private MiniDFSCluster dfsCluster;
 
@@ -111,6 +116,16 @@
         dfs.mkdirs(dest);
         dfs.copyFromLocalFile(src, dest);
 
+        src = new Path(DATA_PATH4);
+        dest = new Path(HDFS_PATH4);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
+
+        src = new Path(DATA_PATH5);
+        dest = new Path(HDFS_PATH5);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
+
         DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
         conf.writeXml(confOutput);
         confOutput.flush();
@@ -138,6 +153,7 @@
         RunJobTestSuite testSuite = new RunJobTestSuite();
         testSuite.setUp();
         boolean onlyEnabled = false;
+        FileSystem dfs = FileSystem.get(testSuite.conf);
 
         if (onlys.size() > 0) {
             onlyEnabled = true;
@@ -153,7 +169,7 @@
                     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));
+                            .toString(), resultFileName, expectedFileName, dfs));
                 }
             }
         }
@@ -193,7 +209,7 @@
 
     private static String jobExtToResExt(String fname) {
         int dot = fname.lastIndexOf('.');
-        return fname.substring(0, dot + 1) + FILE_EXTENSION_OF_RESULTS;
+        return fname.substring(0, dot);
     }
 
     private static boolean isInList(List<String> onlys, String name) {
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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-0
new file mode 100755
index 0000000..f1f1d9b
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-1
new file mode 100755
index 0000000..0fa02c1
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-2
new file mode 100755
index 0000000..542ccae
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/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/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-3
new file mode 100755
index 0000000..1d5d6d9
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-0
new file mode 100755
index 0000000..f1f1d9b
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-1
new file mode 100755
index 0000000..4e7d87a
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-2
new file mode 100755
index 0000000..542ccae
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/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/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-3
new file mode 100755
index 0000000..513f3ff
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation/part-0
new file mode 100755
index 0000000..b5f7ed3
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation/part-1
new file mode 100755
index 0000000..4eca51d
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation/part-2
new file mode 100755
index 0000000..e69de29
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation/part-2
diff --git a/pregelix/pregelix-example/src/test/resources/expected/GraphMutation/part-3 b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation/part-3
new file mode 100755
index 0000000..9446ff7
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/GraphMutation/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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique/part-0
new file mode 100755
index 0000000..902fadf
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique/part-0
@@ -0,0 +1 @@
+4	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique/part-1 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique/part-1
new file mode 100755
index 0000000..ba34424
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique/part-1
@@ -0,0 +1,2 @@
+1	1,2,3,4;
+5	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique/part-2 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique/part-2
new file mode 100755
index 0000000..834e389
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique/part-2
@@ -0,0 +1,2 @@
+2	2,3,4;
+6	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique/part-3 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique/part-3
new file mode 100755
index 0000000..b8e2461
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique/part-3
@@ -0,0 +1,2 @@
+3	3,4;
+7	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2/part-0 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2/part-0
new file mode 100755
index 0000000..902fadf
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2/part-0
@@ -0,0 +1 @@
+4	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2/part-1 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2/part-1
new file mode 100755
index 0000000..b83e1a3
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2/part-2
new file mode 100755
index 0000000..45e2a23
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2/part-2
@@ -0,0 +1,2 @@
+2	2,3;
+6	
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2/part-3 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2/part-3
new file mode 100755
index 0000000..447aa38
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique2/part-3
@@ -0,0 +1 @@
+3	3,4;3,5;
diff --git a/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3/part-0 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3/part-0
new file mode 100755
index 0000000..b4dced4
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3/part-1
new file mode 100755
index 0000000..f554dfe
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3/part-2
new file mode 100755
index 0000000..4df9b8c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3/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/part-3 b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3/part-3
new file mode 100755
index 0000000..5131560
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/MaximalClique3/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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/PageRank/part-0
new file mode 100755
index 0000000..5b07add
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRank/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/PageRank/part-1
new file mode 100755
index 0000000..caf8a4c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRank/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/PageRank/part-2
new file mode 100755
index 0000000..29a9d52
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRank/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/part-3 b/pregelix/pregelix-example/src/test/resources/expected/PageRank/part-3
new file mode 100755
index 0000000..c9bfeb8
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRank/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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal/part-0
new file mode 100755
index 0000000..383076e
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal/part-1
new file mode 100755
index 0000000..0b1a38c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal/part-2
new file mode 100755
index 0000000..ec995b2
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal/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/part-3 b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal/part-3
new file mode 100755
index 0000000..edb7484
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal/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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex/part-0
new file mode 100755
index 0000000..1d26aee2
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex/part-1
new file mode 100755
index 0000000..d013c3e
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex/part-2
new file mode 100755
index 0000000..e8aa0e1
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex/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/part-3 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex/part-3
new file mode 100755
index 0000000..e52970d
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealComplex/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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic/part-0
new file mode 100755
index 0000000..383076e
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic/part-1
new file mode 100755
index 0000000..0b1a38c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic/part-2
new file mode 100755
index 0000000..ec995b2
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic/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/part-3 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic/part-3
new file mode 100755
index 0000000..5593738
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealDynamic/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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner/part-0
new file mode 100755
index 0000000..6fd5f60
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner/part-1
new file mode 100755
index 0000000..0b1a38c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner/part-2
new file mode 100755
index 0000000..69a2803
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner/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/part-3 b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner/part-3
new file mode 100755
index 0000000..7dcb359
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankRealNoCombiner/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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex/part-0
new file mode 100755
index 0000000..6e3ba89
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex/part-1
new file mode 100755
index 0000000..2fcc8eb
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex/part-2
new file mode 100755
index 0000000..c93732c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex/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/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex/part-3
new file mode 100755
index 0000000..c745349
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplex/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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity/part-0
new file mode 100755
index 0000000..8e02e13
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity/part-1
new file mode 100755
index 0000000..cfbb359
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity/part-2
new file mode 100755
index 0000000..8e3ca7c
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity/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/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity/part-3
new file mode 100755
index 0000000..5f83a3d
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ReachibilityRealComplexNoConnectivity/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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths/part-0
new file mode 100755
index 0000000..e8eadcd
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths/part-1
new file mode 100755
index 0000000..c3d200b
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths/part-2
new file mode 100755
index 0000000..779a683
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths/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/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths/part-3
new file mode 100755
index 0000000..57d26c9
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPaths/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/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal/part-0
new file mode 100755
index 0000000..a1cabe2
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal/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/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal/part-1
new file mode 100755
index 0000000..303ed2a
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal/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/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal/part-2
new file mode 100755
index 0000000..0020d25
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal/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/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal/part-3
new file mode 100755
index 0000000..e2657bd
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ShortestPathsReal/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/.part-0.crc b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/.part-0.crc
new file mode 100644
index 0000000..61e7df3
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/.part-0.crc
Binary files differ
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/.part-1.crc b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/.part-1.crc
new file mode 100644
index 0000000..9f34827
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/.part-1.crc
Binary files differ
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/.part-2.crc b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/.part-2.crc
new file mode 100644
index 0000000..cfe7fa2
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/.part-2.crc
Binary files differ
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/.part-3.crc b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/.part-3.crc
new file mode 100644
index 0000000..9959654
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/.part-3.crc
Binary files differ
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/part-0 b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/part-0
new file mode 100755
index 0000000..b36d2b5
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/part-0
@@ -0,0 +1 @@
+4	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/part-1 b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/part-1
new file mode 100755
index 0000000..2812c76
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/part-1
@@ -0,0 +1,2 @@
+1	3
+5	1
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/part-2 b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/part-2
new file mode 100755
index 0000000..25cfb7b
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/part-2
@@ -0,0 +1,2 @@
+2	2
+6	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/part-3 b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/part-3
new file mode 100755
index 0000000..2bd8d38
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/TriangleCounting/part-3
@@ -0,0 +1,2 @@
+3	0
+7	0
diff --git a/pregelix/pregelix-example/src/test/resources/hyracks-deployment.properties b/pregelix/pregelix-example/src/test/resources/hyracks-deployment.properties
deleted file mode 100644
index 9c42b89..0000000
--- a/pregelix/pregelix-example/src/test/resources/hyracks-deployment.properties
+++ /dev/null
@@ -1,2 +0,0 @@
-#cc.bootstrap.class=edu.uci.ics.asterix.hyracks.bootstrap.CCBootstrapImpl
-nc.bootstrap.class=edu.uci.ics.pregelix.runtime.bootstrap.NCBootstrapImpl
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/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 0c09757..0a0a14f 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();
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 1bf6a2b..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();
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);
+    }
+}