Merged hyracks_dev_next into this branch. Fixed a bug in the BTree interior node where deletion would happen incorrectly if that node was the root, and if was last existing key was deleted.

git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_btree_updates_next@663 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-admin-console/pom.xml b/hyracks-admin-console/pom.xml
index ca200e5..080ad69 100644
--- a/hyracks-admin-console/pom.xml
+++ b/hyracks-admin-console/pom.xml
@@ -8,12 +8,12 @@
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-admin-console</artifactId>
   <packaging>war</packaging>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <properties>
diff --git a/hyracks-api/pom.xml b/hyracks-api/pom.xml
index 2301a93..f73c27a 100644
--- a/hyracks-api/pom.xml
+++ b/hyracks-api/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-api</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java
index c739a5e..c07de41 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IApplicationContext.java
@@ -16,8 +16,27 @@
 
 import java.io.Serializable;
 
+/**
+ * Base class of the {@link ICCApplicationContext} and the
+ * {@link INCApplicationContext}.
+ * 
+ * @author vinayakb
+ * 
+ */
 public interface IApplicationContext {
+    /**
+     * Provides the Class Loader that loads classes for this Hyracks Application
+     * at the CC.
+     * 
+     * @return the application {@link ClassLoader}.
+     */
     public ClassLoader getClassLoader();
 
-    public Serializable getDestributedState();
+    /**
+     * Gets the distributed state that is made available to all the Application
+     * Contexts of this application in the cluster.
+     * 
+     * @return
+     */
+    public Serializable getDistributedState();
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IBootstrap.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IBootstrap.java
index efbf0df..19f97b4 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IBootstrap.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IBootstrap.java
@@ -14,8 +14,23 @@
  */
 package edu.uci.ics.hyracks.api.application;
 
+/**
+ * Base class of {@link ICCBootstrap} and {@link INCBootstrap}.
+ * 
+ * @author vinayakb
+ */
 public interface IBootstrap {
+    /**
+     * Method called to start the application at a Hyracks CC or NC node.
+     * 
+     * @throws Exception
+     */
     public void start() throws Exception;
 
+    /**
+     * Method called to shutdown the application at a Hyracks CC or NC node.
+     * 
+     * @throws Exception
+     */
     public void stop() throws Exception;
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
index 0987bd2..2e02101 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
@@ -19,13 +19,51 @@
 import edu.uci.ics.hyracks.api.context.ICCContext;
 import edu.uci.ics.hyracks.api.job.IJobLifecycleListener;
 import edu.uci.ics.hyracks.api.job.IJobSpecificationFactory;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
 
+/**
+ * Application Context at the Cluster Controller for an application.
+ * 
+ * @author vinayakb
+ * 
+ */
 public interface ICCApplicationContext extends IApplicationContext {
+    /**
+     * Sets the state that must be distributed by the infrastructure to all the
+     * NC application contects. Any state set by calling thsi method in the
+     * {@link ICCBootstrap#start()} call is made available to all the
+     * {@link INCApplicationContext} objects at each Node Controller. The state
+     * is then available to be inspected by the application at the NC during or
+     * after the {@link INCBootstrap#start()} call.
+     * 
+     * @param state
+     *            The distributed state
+     */
     public void setDistributedState(Serializable state);
 
+    /**
+     * A factory class specific to this application that may accept incoming
+     * {@link JobSpecification} and produce modified {@link JobSpecification}
+     * that is executed on the cluster. If a {@link IJobSpecificationFactory} is
+     * not set, the incoming {@link JobSpecification} is executed unmodified.
+     * 
+     * @param jobSpecFactory
+     *            - The Job Specification Factory.
+     */
     public void setJobSpecificationFactory(IJobSpecificationFactory jobSpecFactory);
 
+    /**
+     * A listener that listens to Job Lifecycle events at the Cluster
+     * Controller.
+     * 
+     * @param jobLifecycleListener
+     */
     public void addJobLifecycleListener(IJobLifecycleListener jobLifecycleListener);
-    
+
+    /**
+     * Get the Cluster Controller Context.
+     * 
+     * @return The Cluster Controller Context.
+     */
     public ICCContext getCCContext();
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCBootstrap.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCBootstrap.java
index 3acbe5e..e3906ea 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCBootstrap.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCBootstrap.java
@@ -1,5 +1,19 @@
 package edu.uci.ics.hyracks.api.application;
 
+/**
+ * Implemented by the bootstrap class of the application that will manage its
+ * life cycle at the Cluster Controller.
+ * 
+ * @author vinayakb
+ * 
+ */
 public interface ICCBootstrap extends IBootstrap {
+    /**
+     * Called by the infrastructure to set the CC Application Context for the
+     * application. The infrastructure makes this call prior to calling start().
+     * 
+     * @param appCtx
+     *            - The CC application context
+     */
     public void setApplicationContext(ICCApplicationContext appCtx);
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java
index eed5bb3..24a598b 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCApplicationContext.java
@@ -14,18 +14,43 @@
  */
 package edu.uci.ics.hyracks.api.application;
 
-import java.io.Serializable;
-
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
 
+/**
+ * Application Context at the Node Controller for an application.
+ * 
+ * @author vinayakb
+ * 
+ */
 public interface INCApplicationContext extends IApplicationContext {
+    /**
+     * Gets the node Id of the Node Congtroller.
+     * 
+     * @return the Node Id.
+     */
     public String getNodeId();
 
+    /**
+     * Get the Hyracks Root Context.
+     * 
+     * @return The Hyracks Root Context
+     */
     public IHyracksRootContext getRootContext();
 
-    public void setDistributedState(Serializable state);
-
+    /**
+     * Set an object that can be later retrieved by the
+     * {@link #getApplicationObject()} call.
+     * 
+     * @param object
+     *            Application Object
+     */
     public void setApplicationObject(Object object);
 
+    /**
+     * Get the application object previously set by the
+     * {@link #setApplicationObject(Object)} call.
+     * 
+     * @return Application Object
+     */
     public Object getApplicationObject();
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCBootstrap.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCBootstrap.java
index 4aa1ac3..300f7c7 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCBootstrap.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/INCBootstrap.java
@@ -1,5 +1,19 @@
 package edu.uci.ics.hyracks.api.application;
 
+/**
+ * Implemented by the bootstrap class of the application that will manage its
+ * life cycle at a Node Controller.
+ * 
+ * @author vinayakb
+ * 
+ */
 public interface INCBootstrap extends IBootstrap {
+    /**
+     * Called by the infrastructure to set the NC Application Context for the
+     * application. The infrastructure makes this call prior to calling start().
+     * 
+     * @param appCtx
+     *            - The NC application context
+     */
     public void setApplicationContext(INCApplicationContext appCtx);
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
index 7609951..8584d36 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
@@ -14,9 +14,7 @@
  */
 package edu.uci.ics.hyracks.api.context;
 
-import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
 import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
@@ -27,12 +25,4 @@
     public TaskAttemptId getTaskAttemptId();
 
     public ICounterContext getCounterContext();
-
-    public Object lookupGlobalVariable(ActivityId producerActivity, int partition, String varName)
-            throws HyracksDataException;
-
-    public Object lookupLocalVariable(ActivityId producerActivity, int partition, String varName)
-            throws HyracksDataException;
-
-    public void setVariable(String name, Object value) throws HyracksDataException;
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableDescriptor.java
deleted file mode 100644
index b13afe4..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableDescriptor.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.workflow.variables;
-
-import java.io.Serializable;
-
-import edu.uci.ics.hyracks.api.naming.MultipartName;
-
-public final class WorkflowVariableDescriptor implements Serializable {
-    private static final long serialVersionUID = 1L;
-
-    private final MultipartName name;
-
-    private final WorkflowVariableValueScope scope;
-
-    public WorkflowVariableDescriptor(MultipartName name, WorkflowVariableValueScope scope) {
-        this.name = name;
-        this.scope = scope;
-    }
-
-    public MultipartName getName() {
-        return name;
-    }
-
-    public WorkflowVariableValueScope getScope() {
-        return scope;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableValueScope.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableValueScope.java
deleted file mode 100644
index 44ed129..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableValueScope.java
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.workflow.variables;
-
-public enum WorkflowVariableValueScope {
-    LOCAL,
-    GLOBAL,
-}
\ No newline at end of file
diff --git a/hyracks-cli/pom.xml b/hyracks-cli/pom.xml
index ba2c796..1fdc644 100644
--- a/hyracks-cli/pom.xml
+++ b/hyracks-cli/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-cli</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -89,7 +89,7 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-api</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   </dependencies>
diff --git a/hyracks-control-cc/pom.xml b/hyracks-control-cc/pom.xml
index 3fed82f..5b33a3e2 100644
--- a/hyracks-control-cc/pom.xml
+++ b/hyracks-control-cc/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-control-cc</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -27,7 +27,7 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index cbbef2d..7925f0d 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -42,24 +42,22 @@
 import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
 import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.ApplicationDestroyEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.ApplicationStartEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobStatusConditionVariableEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobStatusEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.JobCreateEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.JobStartEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.NodeHeartbeatEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.RegisterNodeEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.RegisterPartitionAvailibilityEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.RegisterPartitionRequestEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.RemoveDeadNodesEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.ReportProfilesEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.TaskCompleteEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.TaskFailureEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.UnregisterNodeEvent;
-import edu.uci.ics.hyracks.control.cc.jobqueue.FutureValue;
-import edu.uci.ics.hyracks.control.cc.jobqueue.JobQueue;
 import edu.uci.ics.hyracks.control.cc.web.WebServer;
+import edu.uci.ics.hyracks.control.cc.work.ApplicationDestroyWork;
+import edu.uci.ics.hyracks.control.cc.work.ApplicationStartWork;
+import edu.uci.ics.hyracks.control.cc.work.GetJobStatusConditionVariableWork;
+import edu.uci.ics.hyracks.control.cc.work.GetJobStatusWork;
+import edu.uci.ics.hyracks.control.cc.work.JobCreateWork;
+import edu.uci.ics.hyracks.control.cc.work.JobStartWork;
+import edu.uci.ics.hyracks.control.cc.work.NodeHeartbeatWork;
+import edu.uci.ics.hyracks.control.cc.work.RegisterNodeWork;
+import edu.uci.ics.hyracks.control.cc.work.RegisterPartitionAvailibilityWork;
+import edu.uci.ics.hyracks.control.cc.work.RegisterPartitionRequestWork;
+import edu.uci.ics.hyracks.control.cc.work.RemoveDeadNodesWork;
+import edu.uci.ics.hyracks.control.cc.work.ReportProfilesWork;
+import edu.uci.ics.hyracks.control.cc.work.TaskCompleteWork;
+import edu.uci.ics.hyracks.control.cc.work.TaskFailureWork;
+import edu.uci.ics.hyracks.control.cc.work.UnregisterNodeWork;
 import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
 import edu.uci.ics.hyracks.control.common.base.IClusterController;
 import edu.uci.ics.hyracks.control.common.base.INodeController;
@@ -72,6 +70,8 @@
 import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
 import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
 import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
+import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.WorkQueue;
 
 public class ClusterControllerService extends AbstractRemoteService implements IClusterController,
         IHyracksClientInterface {
@@ -95,7 +95,7 @@
 
     private final Map<JobId, JobRun> runMap;
 
-    private final JobQueue jobQueue;
+    private final WorkQueue jobQueue;
 
     private final Executor taskExecutor;
 
@@ -117,7 +117,7 @@
         taskExecutor = Executors.newCachedThreadPool();
         webServer = new WebServer(this);
         runMap = new HashMap<JobId, JobRun>();
-        jobQueue = new JobQueue();
+        jobQueue = new WorkQueue();
         this.timer = new Timer(true);
         ccci = new CCClientInterface(this);
         ccContext = new ICCContext() {
@@ -158,7 +158,7 @@
         return runMap;
     }
 
-    public JobQueue getJobQueue() {
+    public WorkQueue getJobQueue() {
         return jobQueue;
     }
 
@@ -185,7 +185,7 @@
     @Override
     public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
         JobId jobId = createJobId();
-        JobCreateEvent jce = new JobCreateEvent(this, jobId, appName, jobSpec, jobFlags);
+        JobCreateWork jce = new JobCreateWork(this, jobId, appName, jobSpec, jobFlags);
         jobQueue.schedule(jce);
         jce.sync();
         return jobId;
@@ -196,7 +196,7 @@
         INodeController nodeController = reg.getNodeController();
         String id = reg.getNodeId();
         NodeControllerState state = new NodeControllerState(nodeController, reg);
-        jobQueue.scheduleAndSync(new RegisterNodeEvent(this, id, state));
+        jobQueue.scheduleAndSync(new RegisterNodeWork(this, id, state));
         nodeController.notifyRegistration(this);
         LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
         NodeParameters params = new NodeParameters();
@@ -209,40 +209,40 @@
     @Override
     public void unregisterNode(INodeController nodeController) throws Exception {
         String id = nodeController.getId();
-        jobQueue.scheduleAndSync(new UnregisterNodeEvent(this, id));
+        jobQueue.scheduleAndSync(new UnregisterNodeWork(this, id));
         LOGGER.log(Level.INFO, "Unregistered INodeController");
     }
 
     @Override
     public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
             throws Exception {
-        TaskCompleteEvent sce = new TaskCompleteEvent(this, jobId, taskId, nodeId);
+        TaskCompleteWork sce = new TaskCompleteWork(this, jobId, taskId, nodeId);
         jobQueue.schedule(sce);
     }
 
     @Override
     public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, Exception exception)
             throws Exception {
-        TaskFailureEvent tfe = new TaskFailureEvent(this, jobId, taskId, nodeId, exception);
+        TaskFailureWork tfe = new TaskFailureWork(this, jobId, taskId, nodeId, exception);
         jobQueue.schedule(tfe);
     }
 
     @Override
     public JobStatus getJobStatus(JobId jobId) throws Exception {
-        GetJobStatusEvent gse = new GetJobStatusEvent(this, jobId);
+        GetJobStatusWork gse = new GetJobStatusWork(this, jobId);
         jobQueue.scheduleAndSync(gse);
         return gse.getStatus();
     }
 
     @Override
     public void start(JobId jobId) throws Exception {
-        JobStartEvent jse = new JobStartEvent(this, jobId);
+        JobStartWork jse = new JobStartWork(this, jobId);
         jobQueue.schedule(jse);
     }
 
     @Override
     public void waitForCompletion(JobId jobId) throws Exception {
-        GetJobStatusConditionVariableEvent e = new GetJobStatusConditionVariableEvent(this, jobId);
+        GetJobStatusConditionVariableWork e = new GetJobStatusConditionVariableWork(this, jobId);
         jobQueue.scheduleAndSync(e);
         IJobStatusConditionVariable var = e.getConditionVariable();
         if (var != null) {
@@ -252,12 +252,12 @@
 
     @Override
     public void reportProfile(String id, List<JobProfile> profiles) throws Exception {
-        jobQueue.schedule(new ReportProfilesEvent(this, profiles));
+        jobQueue.schedule(new ReportProfilesWork(this, profiles));
     }
 
     @Override
     public synchronized void nodeHeartbeat(String id, HeartbeatData hbData) throws Exception {
-        jobQueue.schedule(new NodeHeartbeatEvent(this, id, hbData));
+        jobQueue.schedule(new NodeHeartbeatWork(this, id, hbData));
     }
 
     @Override
@@ -274,14 +274,14 @@
     @Override
     public void destroyApplication(String appName) throws Exception {
         FutureValue fv = new FutureValue();
-        jobQueue.schedule(new ApplicationDestroyEvent(this, appName, fv));
+        jobQueue.schedule(new ApplicationDestroyWork(this, appName, fv));
         fv.get();
     }
 
     @Override
     public void startApplication(final String appName) throws Exception {
         FutureValue fv = new FutureValue();
-        jobQueue.schedule(new ApplicationStartEvent(this, appName, fv));
+        jobQueue.schedule(new ApplicationStartWork(this, appName, fv));
         fv.get();
     }
 
@@ -292,18 +292,18 @@
 
     @Override
     public void registerPartitionProvider(PartitionDescriptor partitionDescriptor) {
-        jobQueue.schedule(new RegisterPartitionAvailibilityEvent(this, partitionDescriptor));
+        jobQueue.schedule(new RegisterPartitionAvailibilityWork(this, partitionDescriptor));
     }
 
     @Override
     public void registerPartitionRequest(PartitionRequest partitionRequest) {
-        jobQueue.schedule(new RegisterPartitionRequestEvent(this, partitionRequest));
+        jobQueue.schedule(new RegisterPartitionRequestWork(this, partitionRequest));
     }
 
     private class DeadNodeSweeper extends TimerTask {
         @Override
         public void run() {
-            jobQueue.schedule(new RemoveDeadNodesEvent(ClusterControllerService.this));
+            jobQueue.schedule(new RemoveDeadNodesWork(ClusterControllerService.this));
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
index 68d794c..14489a6 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
@@ -57,8 +57,8 @@
 import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
 import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
 import edu.uci.ics.hyracks.control.cc.job.TaskClusterAttempt;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.JobCleanupEvent;
 import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
+import edu.uci.ics.hyracks.control.cc.work.JobCleanupWork;
 import edu.uci.ics.hyracks.control.common.job.PartitionState;
 import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
 
@@ -189,7 +189,7 @@
                     + inProgressTaskClusters);
         }
         if (taskClusterRoots.isEmpty() && inProgressTaskClusters.isEmpty()) {
-            ccs.getJobQueue().schedule(new JobCleanupEvent(ccs, jobRun.getJobId(), JobStatus.TERMINATED, null));
+            ccs.getJobQueue().schedule(new JobCleanupWork(ccs, jobRun.getJobId(), JobStatus.TERMINATED, null));
             return;
         }
         startRunnableTaskClusters(taskClusterRoots);
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
index e047212..c10f5ce 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
@@ -28,8 +28,8 @@
 import org.eclipse.jetty.server.handler.AbstractHandler;
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
 public class ApplicationInstallationHandler extends AbstractHandler {
     private ClusterControllerService ccs;
@@ -54,7 +54,7 @@
             }
             final String appName = parts[0];
             if (HttpMethods.PUT.equals(request.getMethod())) {
-                class OutputStreamGetter extends SynchronizableEvent {
+                class OutputStreamGetter extends SynchronizableWork {
                     private OutputStream os;
 
                     @Override
@@ -78,7 +78,7 @@
                     r.os.close();
                 }
             } else if (HttpMethods.GET.equals(request.getMethod())) {
-                class InputStreamGetter extends SynchronizableEvent {
+                class InputStreamGetter extends SynchronizableWork {
                     private InputStream is;
 
                     @Override
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/JobsRESTAPIFunction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/JobsRESTAPIFunction.java
index bd46276..8141dbc 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/JobsRESTAPIFunction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/JobsRESTAPIFunction.java
@@ -18,11 +18,11 @@
 
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobActivityGraphJSONEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobRunJSONEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobSpecificationJSONEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobSummariesJSONEvent;
 import edu.uci.ics.hyracks.control.cc.web.util.IJSONOutputFunction;
+import edu.uci.ics.hyracks.control.cc.work.GetJobActivityGraphJSONWork;
+import edu.uci.ics.hyracks.control.cc.work.GetJobRunJSONWork;
+import edu.uci.ics.hyracks.control.cc.work.GetJobSpecificationJSONWork;
+import edu.uci.ics.hyracks.control.cc.work.GetJobSummariesJSONWork;
 
 public class JobsRESTAPIFunction implements IJSONOutputFunction {
     private ClusterControllerService ccs;
@@ -40,7 +40,7 @@
                     break;
                 }
             case 0: {
-                GetJobSummariesJSONEvent gjse = new GetJobSummariesJSONEvent(ccs);
+                GetJobSummariesJSONWork gjse = new GetJobSummariesJSONWork(ccs);
                 ccs.getJobQueue().scheduleAndSync(gjse);
                 result.put("result", gjse.getSummaries());
                 break;
@@ -50,15 +50,15 @@
                 JobId jobId = JobId.parse(arguments[0]);
 
                 if ("job-specification".equalsIgnoreCase(arguments[1])) {
-                    GetJobSpecificationJSONEvent gjse = new GetJobSpecificationJSONEvent(ccs, jobId);
+                    GetJobSpecificationJSONWork gjse = new GetJobSpecificationJSONWork(ccs, jobId);
                     ccs.getJobQueue().scheduleAndSync(gjse);
                     result.put("result", gjse.getJSON());
                 } else if ("job-activity-graph".equalsIgnoreCase(arguments[1])) {
-                    GetJobActivityGraphJSONEvent gjage = new GetJobActivityGraphJSONEvent(ccs, jobId);
+                    GetJobActivityGraphJSONWork gjage = new GetJobActivityGraphJSONWork(ccs, jobId);
                     ccs.getJobQueue().scheduleAndSync(gjage);
                     result.put("result", gjage.getJSON());
                 } else if ("job-run".equalsIgnoreCase(arguments[1])) {
-                    GetJobRunJSONEvent gjre = new GetJobRunJSONEvent(ccs, jobId);
+                    GetJobRunJSONWork gjre = new GetJobRunJSONWork(ccs, jobId);
                     ccs.getJobQueue().scheduleAndSync(gjre);
                     result.put("result", gjre.getJSON());
                 }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/NodesRESTAPIFunction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/NodesRESTAPIFunction.java
index 9059de7..3e5dd7a 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/NodesRESTAPIFunction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/NodesRESTAPIFunction.java
@@ -17,9 +17,9 @@
 import org.json.JSONObject;
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.GetNodeDetailsJSONEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.GetNodeSummariesJSONEvent;
 import edu.uci.ics.hyracks.control.cc.web.util.IJSONOutputFunction;
+import edu.uci.ics.hyracks.control.cc.work.GetNodeDetailsJSONWork;
+import edu.uci.ics.hyracks.control.cc.work.GetNodeSummariesJSONWork;
 
 public class NodesRESTAPIFunction implements IJSONOutputFunction {
     private ClusterControllerService ccs;
@@ -34,12 +34,12 @@
         switch (arguments.length) {
             case 1: {
                 if ("".equals(arguments[0])) {
-                    GetNodeSummariesJSONEvent gnse = new GetNodeSummariesJSONEvent(ccs);
+                    GetNodeSummariesJSONWork gnse = new GetNodeSummariesJSONWork(ccs);
                     ccs.getJobQueue().scheduleAndSync(gnse);
                     result.put("result", gnse.getSummaries());
                 } else {
                     String nodeId = arguments[0];
-                    GetNodeDetailsJSONEvent gnde = new GetNodeDetailsJSONEvent(ccs, nodeId);
+                    GetNodeDetailsJSONWork gnde = new GetNodeDetailsJSONWork(ccs, nodeId);
                     ccs.getJobQueue().scheduleAndSync(gnde);
                     result.put("result", gnde.getDetail());
                 }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/AbstractTaskLifecycleEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/AbstractTaskLifecycleWork.java
similarity index 90%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/AbstractTaskLifecycleEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/AbstractTaskLifecycleWork.java
index 0028b6f..b240da4 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/AbstractTaskLifecycleEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/AbstractTaskLifecycleWork.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import java.util.List;
 import java.util.Map;
@@ -29,15 +29,15 @@
 import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
 import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
 import edu.uci.ics.hyracks.control.cc.job.TaskClusterAttempt;
-import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
-public abstract class AbstractTaskLifecycleEvent extends AbstractEvent {
+public abstract class AbstractTaskLifecycleWork extends AbstractWork {
     protected final ClusterControllerService ccs;
     protected final JobId jobId;
     protected final TaskAttemptId taId;
     protected final String nodeId;
 
-    public AbstractTaskLifecycleEvent(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId) {
+    public AbstractTaskLifecycleWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId) {
         this.ccs = ccs;
         this.jobId = jobId;
         this.taId = taId;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
similarity index 85%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
index 050c6ed..94873d3 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
@@ -12,26 +12,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import java.util.ArrayList;
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
-import edu.uci.ics.hyracks.control.cc.jobqueue.FutureValue;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
 import edu.uci.ics.hyracks.control.cc.remote.ops.ApplicationDestroyer;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+import edu.uci.ics.hyracks.control.common.work.FutureValue;
 
-public class ApplicationDestroyEvent extends AbstractEvent {
+public class ApplicationDestroyWork extends AbstractWork {
     private final ClusterControllerService ccs;
     private final String appName;
     private FutureValue fv;
 
-    public ApplicationDestroyEvent(ClusterControllerService ccs, String appName, FutureValue fv) {
+    public ApplicationDestroyWork(ClusterControllerService ccs, String appName, FutureValue fv) {
         this.ccs = ccs;
         this.appName = appName;
         this.fv = fv;
@@ -58,7 +58,7 @@
                     fv.setException(e);
                     return;
                 }
-                ccs.getJobQueue().schedule(new AbstractEvent() {
+                ccs.getJobQueue().schedule(new AbstractWork() {
                     @Override
                     public void run() {
                         try {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationStartEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
similarity index 86%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationStartEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
index ab05d0f..a65703b 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationStartEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -20,19 +20,19 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
-import edu.uci.ics.hyracks.control.cc.jobqueue.FutureValue;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
 import edu.uci.ics.hyracks.control.cc.remote.ops.ApplicationStarter;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+import edu.uci.ics.hyracks.control.common.work.FutureValue;
 
-public class ApplicationStartEvent extends AbstractEvent {
+public class ApplicationStartWork extends AbstractWork {
     private final ClusterControllerService ccs;
     private final String appName;
     private final FutureValue fv;
 
-    public ApplicationStartEvent(ClusterControllerService ccs, String appName, FutureValue fv) {
+    public ApplicationStartWork(ClusterControllerService ccs, String appName, FutureValue fv) {
         this.ccs = ccs;
         this.appName = appName;
         this.fv = fv;
@@ -48,7 +48,7 @@
         try {
             appCtx.initializeClassPath();
             appCtx.initialize();
-            final byte[] distributedState = JavaSerializationUtils.serialize(appCtx.getDestributedState());
+            final byte[] distributedState = JavaSerializationUtils.serialize(appCtx.getDistributedState());
             final boolean deployHar = appCtx.containsHar();
             List<RemoteOp<Void>> opList = new ArrayList<RemoteOp<Void>>();
             for (final String nodeId : ccs.getNodeMap().keySet()) {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobActivityGraphJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobActivityGraphJSONWork.java
similarity index 81%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobActivityGraphJSONEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobActivityGraphJSONWork.java
index 14b56df..28823c2 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobActivityGraphJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobActivityGraphJSONWork.java
@@ -12,21 +12,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import org.json.JSONObject;
 
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class GetJobActivityGraphJSONEvent extends SynchronizableEvent {
+public class GetJobActivityGraphJSONWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final JobId jobId;
     private JSONObject json;
 
-    public GetJobActivityGraphJSONEvent(ClusterControllerService ccs, JobId jobId) {
+    public GetJobActivityGraphJSONWork(ClusterControllerService ccs, JobId jobId) {
         this.ccs = ccs;
         this.jobId = jobId;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobProfileJSONWork.java
similarity index 82%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobProfileJSONWork.java
index 4355962..513cb93 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobProfileJSONWork.java
@@ -12,21 +12,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import org.json.JSONObject;
 
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class GetJobProfileJSONEvent extends SynchronizableEvent {
+public class GetJobProfileJSONWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final JobId jobId;
     private JSONObject json;
 
-    public GetJobProfileJSONEvent(ClusterControllerService ccs, JobId jobId) {
+    public GetJobProfileJSONWork(ClusterControllerService ccs, JobId jobId) {
         this.ccs = ccs;
         this.jobId = jobId;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobRunJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobRunJSONWork.java
similarity index 82%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobRunJSONEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobRunJSONWork.java
index 3e19a70..9de7613 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobRunJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobRunJSONWork.java
@@ -12,21 +12,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import org.json.JSONObject;
 
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class GetJobRunJSONEvent extends SynchronizableEvent {
+public class GetJobRunJSONWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final JobId jobId;
     private JSONObject json;
 
-    public GetJobRunJSONEvent(ClusterControllerService ccs, JobId jobId) {
+    public GetJobRunJSONWork(ClusterControllerService ccs, JobId jobId) {
         this.ccs = ccs;
         this.jobId = jobId;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSpecificationJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSpecificationJSONWork.java
similarity index 81%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSpecificationJSONEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSpecificationJSONWork.java
index db7e5a7..7e16c9c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSpecificationJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSpecificationJSONWork.java
@@ -12,21 +12,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import org.json.JSONObject;
 
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class GetJobSpecificationJSONEvent extends SynchronizableEvent {
+public class GetJobSpecificationJSONWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final JobId jobId;
     private JSONObject json;
 
-    public GetJobSpecificationJSONEvent(ClusterControllerService ccs, JobId jobId) {
+    public GetJobSpecificationJSONWork(ClusterControllerService ccs, JobId jobId) {
         this.ccs = ccs;
         this.jobId = jobId;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusConditionVariableEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusConditionVariableWork.java
similarity index 79%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusConditionVariableEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusConditionVariableWork.java
index 357d62b..67eca4a 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusConditionVariableEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusConditionVariableWork.java
@@ -12,19 +12,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class GetJobStatusConditionVariableEvent extends SynchronizableEvent {
+public class GetJobStatusConditionVariableWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final JobId jobId;
     private IJobStatusConditionVariable cVar;
 
-    public GetJobStatusConditionVariableEvent(ClusterControllerService ccs, JobId jobId) {
+    public GetJobStatusConditionVariableWork(ClusterControllerService ccs, JobId jobId) {
         this.ccs = ccs;
         this.jobId = jobId;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusWork.java
similarity index 82%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusWork.java
index fbe6e4c..cd29207 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusWork.java
@@ -12,20 +12,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class GetJobStatusEvent extends SynchronizableEvent {
+public class GetJobStatusWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final JobId jobId;
     private JobStatus status;
 
-    public GetJobStatusEvent(ClusterControllerService ccs, JobId jobId) {
+    public GetJobStatusWork(ClusterControllerService ccs, JobId jobId) {
         this.ccs = ccs;
         this.jobId = jobId;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSummariesJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSummariesJSONWork.java
similarity index 85%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSummariesJSONEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSummariesJSONWork.java
index eda4338..64b9058 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSummariesJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSummariesJSONWork.java
@@ -12,20 +12,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import org.json.JSONArray;
 import org.json.JSONObject;
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class GetJobSummariesJSONEvent extends SynchronizableEvent {
+public class GetJobSummariesJSONWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private JSONArray summaries;
 
-    public GetJobSummariesJSONEvent(ClusterControllerService ccs) {
+    public GetJobSummariesJSONWork(ClusterControllerService ccs) {
         this.ccs = ccs;
     }
 
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeDetailsJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
similarity index 81%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeDetailsJSONEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
index 284ec35..4a1f108 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeDetailsJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
@@ -12,20 +12,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import org.json.JSONObject;
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class GetNodeDetailsJSONEvent extends SynchronizableEvent {
+public class GetNodeDetailsJSONWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final String nodeId;
     private JSONObject detail;
 
-    public GetNodeDetailsJSONEvent(ClusterControllerService ccs, String nodeId) {
+    public GetNodeDetailsJSONWork(ClusterControllerService ccs, String nodeId) {
         this.ccs = ccs;
         this.nodeId = nodeId;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeSummariesJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeSummariesJSONWork.java
similarity index 81%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeSummariesJSONEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeSummariesJSONWork.java
index 512bce2..6ac4ac2 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeSummariesJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeSummariesJSONWork.java
@@ -12,19 +12,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import org.json.JSONArray;
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class GetNodeSummariesJSONEvent extends SynchronizableEvent {
+public class GetNodeSummariesJSONWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private JSONArray summaries;
 
-    public GetNodeSummariesJSONEvent(ClusterControllerService ccs) {
+    public GetNodeSummariesJSONWork(ClusterControllerService ccs) {
         this.ccs = ccs;
     }
 
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCleanupEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
similarity index 89%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCleanupEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
index 4274914..0d87b0d 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCleanupEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import java.util.Set;
 
@@ -23,17 +23,17 @@
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
 import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
 import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
 import edu.uci.ics.hyracks.control.cc.remote.ops.JobCompleteNotifier;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
-public class JobCleanupEvent extends AbstractEvent {
+public class JobCleanupWork extends AbstractWork {
     private ClusterControllerService ccs;
     private JobId jobId;
     private JobStatus status;
     private Exception exception;
 
-    public JobCleanupEvent(ClusterControllerService ccs, JobId jobId, JobStatus status, Exception exception) {
+    public JobCleanupWork(ClusterControllerService ccs, JobId jobId, JobStatus status, Exception exception) {
         this.ccs = ccs;
         this.jobId = jobId;
         this.status = status;
@@ -63,7 +63,7 @@
                         e.printStackTrace();
                     }
                 }
-                ccs.getJobQueue().schedule(new AbstractEvent() {
+                ccs.getJobQueue().schedule(new AbstractWork() {
                     @Override
                     public void run() {
                         CCApplicationContext appCtx = ccs.getApplicationMap().get(
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCreateEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java
similarity index 90%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCreateEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java
index 5fadb00..7ede593 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCreateEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import java.util.EnumSet;
 
@@ -29,17 +29,17 @@
 import edu.uci.ics.hyracks.control.cc.job.JobActivityGraphBuilder;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
 import edu.uci.ics.hyracks.control.cc.job.PlanUtils;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 import edu.uci.ics.hyracks.control.cc.scheduler.JobScheduler;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class JobCreateEvent extends SynchronizableEvent {
+public class JobCreateWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final byte[] jobSpec;
     private final EnumSet<JobFlag> jobFlags;
     private final JobId jobId;
     private final String appName;
 
-    public JobCreateEvent(ClusterControllerService ccs, JobId jobId, String appName, byte[] jobSpec,
+    public JobCreateWork(ClusterControllerService ccs, JobId jobId, String appName, byte[] jobSpec,
             EnumSet<JobFlag> jobFlags) {
         this.jobId = jobId;
         this.ccs = ccs;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
similarity index 79%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
index 8f6357c..0fa5710 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
@@ -12,19 +12,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class JobStartEvent extends SynchronizableEvent {
+public class JobStartWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final JobId jobId;
 
-    public JobStartEvent(ClusterControllerService ccs, JobId jobId) {
+    public JobStartWork(ClusterControllerService ccs, JobId jobId) {
         this.ccs = ccs;
         this.jobId = jobId;
     }
@@ -42,7 +42,7 @@
         try {
             run.getScheduler().startJob();
         } catch (Exception e) {
-            ccs.getJobQueue().schedule(new JobCleanupEvent(ccs, run.getJobId(), JobStatus.FAILURE, e));
+            ccs.getJobQueue().schedule(new JobCleanupWork(ccs, run.getJobId(), JobStatus.FAILURE, e));
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NodeHeartbeatWork.java
similarity index 82%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NodeHeartbeatWork.java
index 98d17ad..6b139a5 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NodeHeartbeatWork.java
@@ -12,22 +12,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import java.util.Map;
 import java.util.logging.Level;
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class NodeHeartbeatEvent extends SynchronizableEvent {
+public class NodeHeartbeatWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final String nodeId;
     private final HeartbeatData hbData;
 
-    public NodeHeartbeatEvent(ClusterControllerService ccs, String nodeId, HeartbeatData hbData) {
+    public NodeHeartbeatWork(ClusterControllerService ccs, String nodeId, HeartbeatData hbData) {
         this.ccs = ccs;
         this.nodeId = nodeId;
         this.hbData = hbData;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterNodeEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
similarity index 85%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterNodeEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
index 5f9377b..392708b 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterNodeEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import java.util.HashSet;
 import java.util.Map;
@@ -20,14 +20,14 @@
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class RegisterNodeEvent extends SynchronizableEvent {
+public class RegisterNodeWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final String nodeId;
     private final NodeControllerState state;
 
-    public RegisterNodeEvent(ClusterControllerService ccs, String nodeId, NodeControllerState state) {
+    public RegisterNodeWork(ClusterControllerService ccs, String nodeId, NodeControllerState state) {
         this.ccs = ccs;
         this.nodeId = nodeId;
         this.state = state;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionAvailibilityEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java
similarity index 86%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionAvailibilityEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java
index 3f49b97..ec9a356 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionAvailibilityEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import java.util.List;
 
@@ -20,17 +20,17 @@
 import edu.uci.ics.hyracks.api.util.Pair;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
 import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
 import edu.uci.ics.hyracks.control.cc.partitions.PartitionUtils;
 import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
 import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
-public class RegisterPartitionAvailibilityEvent extends AbstractEvent {
+public class RegisterPartitionAvailibilityWork extends AbstractWork {
     private final ClusterControllerService ccs;
     private final PartitionDescriptor partitionDescriptor;
 
-    public RegisterPartitionAvailibilityEvent(ClusterControllerService ccs, PartitionDescriptor partitionDescriptor) {
+    public RegisterPartitionAvailibilityWork(ClusterControllerService ccs, PartitionDescriptor partitionDescriptor) {
         this.ccs = ccs;
         this.partitionDescriptor = partitionDescriptor;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionRequestEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionRequestWork.java
similarity index 85%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionRequestEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionRequestWork.java
index bac8dd1..a7a36cb 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionRequestEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterPartitionRequestWork.java
@@ -12,23 +12,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
 import edu.uci.ics.hyracks.api.util.Pair;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
 import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
 import edu.uci.ics.hyracks.control.cc.partitions.PartitionUtils;
 import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
 import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
-public class RegisterPartitionRequestEvent extends AbstractEvent {
+public class RegisterPartitionRequestWork extends AbstractWork {
     private final ClusterControllerService ccs;
     private final PartitionRequest partitionRequest;
 
-    public RegisterPartitionRequestEvent(ClusterControllerService ccs, PartitionRequest partitionRequest) {
+    public RegisterPartitionRequestWork(ClusterControllerService ccs, PartitionRequest partitionRequest) {
         this.ccs = ccs;
         this.partitionRequest = partitionRequest;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RemoveDeadNodesWork.java
similarity index 90%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RemoveDeadNodesWork.java
index b812594..889faa2 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RemoveDeadNodesWork.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import java.util.HashSet;
 import java.util.Map;
@@ -24,14 +24,14 @@
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
-public class RemoveDeadNodesEvent extends AbstractEvent {
-    private static Logger LOGGER = Logger.getLogger(RemoveDeadNodesEvent.class.getName());
+public class RemoveDeadNodesWork extends AbstractWork {
+    private static Logger LOGGER = Logger.getLogger(RemoveDeadNodesWork.class.getName());
 
     private final ClusterControllerService ccs;
 
-    public RemoveDeadNodesEvent(ClusterControllerService ccs) {
+    public RemoveDeadNodesWork(ClusterControllerService ccs) {
         this.ccs = ccs;
     }
 
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ReportProfilesEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportProfilesWork.java
similarity index 83%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ReportProfilesEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportProfilesWork.java
index bd82f75..df12ee9 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ReportProfilesEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportProfilesWork.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import java.util.List;
 import java.util.Map;
@@ -21,14 +21,14 @@
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
 import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
-public class ReportProfilesEvent extends AbstractEvent {
+public class ReportProfilesWork extends AbstractWork {
     private final ClusterControllerService ccs;
     private final List<JobProfile> profiles;
 
-    public ReportProfilesEvent(ClusterControllerService ccs, List<JobProfile> profiles) {
+    public ReportProfilesWork(ClusterControllerService ccs, List<JobProfile> profiles) {
         this.ccs = ccs;
         this.profiles = profiles;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskCompleteEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskCompleteWork.java
similarity index 85%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskCompleteEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskCompleteWork.java
index 19fd95b..0d12c65 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskCompleteEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskCompleteWork.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
@@ -21,8 +21,8 @@
 import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
 import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
 
-public class TaskCompleteEvent extends AbstractTaskLifecycleEvent {
-    public TaskCompleteEvent(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId) {
+public class TaskCompleteWork extends AbstractTaskLifecycleWork {
+    public TaskCompleteWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId) {
         super(ccs, jobId, taId, nodeId);
     }
 
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskFailureEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
similarity index 85%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskFailureEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
index 8b7e821..31f0872 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskFailureEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.job.JobId;
@@ -20,10 +20,10 @@
 import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
 import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
 
-public class TaskFailureEvent extends AbstractTaskLifecycleEvent {
+public class TaskFailureWork extends AbstractTaskLifecycleWork {
     private final Exception exception;
 
-    public TaskFailureEvent(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId,
+    public TaskFailureWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId,
             Exception exception) {
         super(ccs, jobId, taId, nodeId);
         this.exception = exception;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/UnregisterNodeWork.java
similarity index 79%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/UnregisterNodeWork.java
index b89de07..32ddba9 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/UnregisterNodeWork.java
@@ -12,19 +12,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job.manager.events;
+package edu.uci.ics.hyracks.control.cc.work;
 
 import java.util.Map;
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class UnregisterNodeEvent extends SynchronizableEvent {
+public class UnregisterNodeWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final String nodeId;
 
-    public UnregisterNodeEvent(ClusterControllerService ccs, String nodeId) {
+    public UnregisterNodeWork(ClusterControllerService ccs, String nodeId) {
         this.ccs = ccs;
         this.nodeId = nodeId;
     }
diff --git a/hyracks-control-common/pom.xml b/hyracks-control-common/pom.xml
index 3ff12d7..fc7a2d5 100644
--- a/hyracks-control-common/pom.xml
+++ b/hyracks-control-common/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-control-common</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -27,7 +27,7 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-api</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
index c02c7ad..d2a06be 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
@@ -195,7 +195,7 @@
     }
 
     @Override
-    public Serializable getDestributedState() {
+    public Serializable getDistributedState() {
         return distributedState;
     }
 
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
index d12d037..6afa43e 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.hyracks.control.common.controllers;
 
+import java.util.List;
+
 import org.kohsuke.args4j.Option;
 
 public class CCConfig {
@@ -34,4 +36,19 @@
 
     @Option(name = "-default-max-job-attempts", usage = "Sets the default number of job attempts allowed if not specified in the job specification. (default: 5)")
     public int defaultMaxJobAttempts = 5;
+
+    public void toCommandLine(List<String> cList) {
+        cList.add("-port");
+        cList.add(String.valueOf(port));
+        cList.add("-http-port");
+        cList.add(String.valueOf(httpPort));
+        cList.add("-heartbeat-period");
+        cList.add(String.valueOf(heartbeatPeriod));
+        cList.add("-max-heartbeat-lapse-periods");
+        cList.add(String.valueOf(maxHeartbeatLapsePeriods));
+        cList.add("-profile-dump-period");
+        cList.add(String.valueOf(profileDumpPeriod));
+        cList.add("-default-max-job-attempts");
+        cList.add(String.valueOf(defaultMaxJobAttempts));
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index c887223..02ce0f8 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -15,6 +15,7 @@
 package edu.uci.ics.hyracks.control.common.controllers;
 
 import java.io.Serializable;
+import java.util.List;
 
 import org.kohsuke.args4j.Option;
 
@@ -47,4 +48,27 @@
 
     @Option(name = "-dcache-client-path", usage = "Sets the path to store the files retrieved from the DCache server (default /tmp/dcache-client)")
     public String dcacheClientPath = "/tmp/dcache-client";
+
+    public void toCommandLine(List<String> cList) {
+        cList.add("-cc-host");
+        cList.add(ccHost);
+        cList.add("-cc-port");
+        cList.add(String.valueOf(ccPort));
+        cList.add("-node-id");
+        cList.add(nodeId);
+        cList.add("-data-ip-address");
+        cList.add(dataIPAddress);
+        cList.add("-frame-size");
+        cList.add(String.valueOf(frameSize));
+        cList.add("-iodevices");
+        cList.add(ioDevices);
+        cList.add("-dcache-client-servers");
+        cList.add(dcacheClientServers);
+        if (dcacheClientServerLocal != null) {
+            cList.add("-dcache-client-server-local");
+            cList.add(dcacheClientServerLocal);
+        }
+        cList.add("-dcache-client-path");
+        cList.add(dcacheClientPath);
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/AbstractEvent.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/AbstractWork.java
similarity index 86%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/AbstractEvent.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/AbstractWork.java
index 5fcd56a..4ead100 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/AbstractEvent.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/AbstractWork.java
@@ -12,11 +12,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.jobqueue;
+package edu.uci.ics.hyracks.control.common.work;
 
 import java.util.logging.Level;
 
-public abstract class AbstractEvent implements Runnable {
+public abstract class AbstractWork implements Runnable {
     public Level logLevel() {
         return Level.INFO;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/FutureValue.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/FutureValue.java
similarity index 96%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/FutureValue.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/FutureValue.java
index 25378c6..0e4ccc0 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/FutureValue.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/FutureValue.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.jobqueue;
+package edu.uci.ics.hyracks.control.common.work;
 
 public class FutureValue {
     private boolean done;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableEvent.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/SynchronizableWork.java
similarity index 89%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableEvent.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/SynchronizableWork.java
index 88c9097..e94086f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableEvent.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/SynchronizableWork.java
@@ -12,9 +12,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.jobqueue;
+package edu.uci.ics.hyracks.control.common.work;
 
-public abstract class SynchronizableEvent extends AbstractEvent {
+public abstract class SynchronizableWork extends AbstractWork {
     private boolean done;
 
     private Exception e;
@@ -45,7 +45,7 @@
             try {
                 wait();
             } catch (InterruptedException e) {
-                e.printStackTrace();
+                throw e;
             }
         }
         if (e != null) {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/JobQueue.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java
similarity index 78%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/JobQueue.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java
index 84a844c..2fc172e 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/JobQueue.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java
@@ -12,31 +12,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.jobqueue;
+package edu.uci.ics.hyracks.control.common.work;
 
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.logging.Logger;
 
-public class JobQueue {
-    private static final Logger LOGGER = Logger.getLogger(JobQueue.class.getName());
+public class WorkQueue {
+    private static final Logger LOGGER = Logger.getLogger(WorkQueue.class.getName());
 
-    private final LinkedBlockingQueue<AbstractEvent> queue;
+    private final LinkedBlockingQueue<AbstractWork> queue;
     private final JobThread thread;
 
-    public JobQueue() {
-        queue = new LinkedBlockingQueue<AbstractEvent>();
+    public WorkQueue() {
+        queue = new LinkedBlockingQueue<AbstractWork>();
         thread = new JobThread();
         thread.start();
     }
 
-    public void schedule(AbstractEvent event) {
+    public void schedule(AbstractWork event) {
         if (LOGGER.isLoggable(event.logLevel())) {
             LOGGER.info("Scheduling: " + event);
         }
         queue.offer(event);
     }
 
-    public void scheduleAndSync(SynchronizableEvent sRunnable) throws Exception {
+    public void scheduleAndSync(SynchronizableWork sRunnable) throws Exception {
         schedule(sRunnable);
         sRunnable.sync();
     }
diff --git a/hyracks-control-nc/pom.xml b/hyracks-control-nc/pom.xml
index 3c98365..64a409c 100644
--- a/hyracks-control-nc/pom.xml
+++ b/hyracks-control-nc/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-control-nc</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -33,7 +33,7 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index 2176dda..cf6b4ed 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -38,7 +38,6 @@
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.naming.MultipartName;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
 import edu.uci.ics.hyracks.api.resources.IDeallocatable;
 import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
@@ -69,8 +68,6 @@
 
     private final Map<String, Counter> counterMap;
 
-    private final Map<MultipartName, Object> localVariableMap;
-
     private final DefaultDeallocatableRegistry deallocatableRegistry;
 
     private final IWorkspaceFileFactory fileFactory;
@@ -84,7 +81,6 @@
         taskStateMap = new HashMap<TaskId, ITaskState>();
         taskMap = new HashMap<TaskAttemptId, Task>();
         counterMap = new HashMap<String, Counter>();
-        localVariableMap = new HashMap<MultipartName, Object>();
         deallocatableRegistry = new DefaultDeallocatableRegistry();
         fileFactory = new WorkspaceFileFactory(this, (IOManager) appCtx.getRootContext().getIOManager());
     }
@@ -113,17 +109,6 @@
         return taskMap;
     }
 
-    public synchronized Object lookupLocalVariable(MultipartName name) throws HyracksDataException {
-        if (!localVariableMap.containsKey(name)) {
-            throw new HyracksDataException("Unknown variable: " + name);
-        }
-        return localVariableMap.get(name);
-    }
-
-    public synchronized void setLocalVariable(MultipartName name, Object value) {
-        localVariableMap.put(name, value);
-    }
-
     private final class OperatorEnvironmentImpl implements IOperatorEnvironment {
         private final String nodeId;
 
@@ -154,12 +139,14 @@
         taskMap.remove(task);
         TaskProfile taskProfile = new TaskProfile(task.getTaskAttemptId());
         task.dumpProfile(taskProfile);
-        nodeController.notifyTaskComplete(jobId, task.getTaskAttemptId(), taskProfile);
+        nodeController.getClusterController().notifyTaskComplete(jobId, task.getTaskAttemptId(),
+                nodeController.getId(), taskProfile);
     }
 
-    public synchronized void notifyTaskFailed(Task task, Exception exception) {
+    public synchronized void notifyTaskFailed(Task task, Exception exception) throws Exception {
         taskMap.remove(task);
-        nodeController.notifyTaskFailed(jobId, task.getTaskAttemptId(), exception);
+        nodeController.getClusterController().notifyTaskFailure(jobId, task.getTaskAttemptId(), nodeController.getId(),
+                exception);
     }
 
     public NodeControllerService getNodeController() {
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 28cb8be..9a9aee2 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -15,9 +15,6 @@
 package edu.uci.ics.hyracks.control.nc;
 
 import java.io.File;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryMXBean;
 import java.lang.management.MemoryUsage;
@@ -25,12 +22,10 @@
 import java.lang.management.RuntimeMXBean;
 import java.lang.management.ThreadMXBean;
 import java.net.InetAddress;
-import java.net.InetSocketAddress;
 import java.rmi.registry.LocateRegistry;
 import java.rmi.registry.Registry;
 import java.text.MessageFormat;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.Hashtable;
 import java.util.List;
 import java.util.Map;
@@ -44,39 +39,15 @@
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-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.INCApplicationContext;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
-import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
-import edu.uci.ics.hyracks.api.comm.PartitionChannel;
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
 import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.IActivity;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
-import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.IODeviceHandle;
-import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
-import edu.uci.ics.hyracks.api.job.JobActivityGraph;
 import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.naming.MultipartName;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
 import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
-import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
 import edu.uci.ics.hyracks.control.common.base.IClusterController;
 import edu.uci.ics.hyracks.control.common.base.INodeController;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
@@ -86,17 +57,20 @@
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
 import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
 import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.control.common.job.profiling.om.JobletProfile;
-import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
+import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.WorkQueue;
 import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.control.nc.net.ConnectionManager;
-import edu.uci.ics.hyracks.control.nc.net.NetworkInputChannel;
-import edu.uci.ics.hyracks.control.nc.partitions.MaterializedPartitionWriter;
 import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
-import edu.uci.ics.hyracks.control.nc.partitions.PipelinedPartition;
-import edu.uci.ics.hyracks.control.nc.partitions.ReceiveSideMaterializingCollector;
 import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
+import edu.uci.ics.hyracks.control.nc.work.AbortTasksWork;
+import edu.uci.ics.hyracks.control.nc.work.BuildJobProfilesWork;
+import edu.uci.ics.hyracks.control.nc.work.CleanupJobWork;
+import edu.uci.ics.hyracks.control.nc.work.CreateApplicationWork;
+import edu.uci.ics.hyracks.control.nc.work.DestroyApplicationWork;
+import edu.uci.ics.hyracks.control.nc.work.ReportPartitionAvailabilityWork;
+import edu.uci.ics.hyracks.control.nc.work.StartTasksWork;
 
 public class NodeControllerService extends AbstractRemoteService implements INodeController {
     private static Logger LOGGER = Logger.getLogger(NodeControllerService.class.getName());
@@ -113,6 +87,8 @@
 
     private final ConnectionManager connectionManager;
 
+    private final WorkQueue queue;
+
     private final Timer timer;
 
     private IClusterController ccs;
@@ -147,6 +123,7 @@
         partitionManager = new PartitionManager(this);
         connectionManager.setPartitionRequestListener(partitionManager);
 
+        queue = new WorkQueue();
         jobletMap = new Hashtable<JobId, Joblet>();
         timer = new Timer(true);
         serverCtx = new ServerContext(ServerContext.ServerType.NODE_CONTROLLER, new File(new File(
@@ -206,6 +183,18 @@
         return id;
     }
 
+    public ServerContext getServerContext() {
+        return serverCtx;
+    }
+
+    public Map<String, NCApplicationContext> getApplications() {
+        return applications;
+    }
+
+    public Map<JobId, Joblet> getJobletMap() {
+        return jobletMap;
+    }
+
     public ConnectionManager getConnectionManager() {
         return connectionManager;
     }
@@ -218,6 +207,63 @@
         return ccs;
     }
 
+    public NodeParameters getNodeParameters() {
+        return nodeParameters;
+    }
+
+    public Executor getExecutor() {
+        return executor;
+    }
+
+    @Override
+    public void startTasks(String appName, final JobId jobId, byte[] jagBytes,
+            List<TaskAttemptDescriptor> taskDescriptors,
+            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap, byte[] ctxVarBytes) throws Exception {
+        StartTasksWork stw = new StartTasksWork(this, appName, jobId, jagBytes, taskDescriptors, connectorPoliciesMap);
+        queue.scheduleAndSync(stw);
+    }
+
+    @Override
+    public void cleanUpJob(JobId jobId) throws Exception {
+        CleanupJobWork cjw = new CleanupJobWork(this, jobId);
+        queue.scheduleAndSync(cjw);
+    }
+
+    @Override
+    public void notifyRegistration(IClusterController ccs) throws Exception {
+        this.ccs = ccs;
+    }
+
+    @Override
+    public NCConfig getConfiguration() throws Exception {
+        return ncConfig;
+    }
+
+    @Override
+    public synchronized void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception {
+        AbortTasksWork atw = new AbortTasksWork(this, jobId, tasks);
+        queue.scheduleAndSync(atw);
+    }
+
+    @Override
+    public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
+            throws Exception {
+        CreateApplicationWork caw = new CreateApplicationWork(this, appName, deployHar, serializedDistributedState);
+        queue.scheduleAndSync(caw);
+    }
+
+    @Override
+    public void destroyApplication(String appName) throws Exception {
+        DestroyApplicationWork daw = new DestroyApplicationWork(this, appName);
+        queue.scheduleAndSync(daw);
+    }
+
+    @Override
+    public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception {
+        ReportPartitionAvailabilityWork rpaw = new ReportPartitionAvailabilityWork(this, pid, networkAddress);
+        queue.scheduleAndSync(rpaw);
+    }
+
     private static InetAddress getIpAddress(NCConfig ncConfig) throws Exception {
         String ipaddrStr = ncConfig.dataIPAddress;
         ipaddrStr = ipaddrStr.trim();
@@ -235,184 +281,6 @@
         return InetAddress.getByAddress(ipBytes);
     }
 
-    @Override
-    public void startTasks(String appName, final JobId jobId, byte[] jagBytes,
-            List<TaskAttemptDescriptor> taskDescriptors,
-            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap, byte[] ctxVarBytes) throws Exception {
-        try {
-            NCApplicationContext appCtx = applications.get(appName);
-            final JobActivityGraph plan = (JobActivityGraph) appCtx.deserialize(jagBytes);
-            Map<MultipartName, Object> ctxVarMap = (Map<MultipartName, Object>) appCtx.deserialize(ctxVarBytes);
-
-            IRecordDescriptorProvider rdp = new IRecordDescriptorProvider() {
-                @Override
-                public RecordDescriptor getOutputRecordDescriptor(OperatorDescriptorId opId, int outputIndex) {
-                    return plan.getJobSpecification().getOperatorOutputRecordDescriptor(opId, outputIndex);
-                }
-
-                @Override
-                public RecordDescriptor getInputRecordDescriptor(OperatorDescriptorId opId, int inputIndex) {
-                    return plan.getJobSpecification().getOperatorInputRecordDescriptor(opId, inputIndex);
-                }
-            };
-
-            final Joblet joblet = getOrCreateLocalJoblet(jobId, appCtx);
-
-            for (TaskAttemptDescriptor td : taskDescriptors) {
-                TaskAttemptId taId = td.getTaskAttemptId();
-                TaskId tid = taId.getTaskId();
-                IActivity han = plan.getActivityNodeMap().get(tid.getActivityId());
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Initializing " + taId + " -> " + han);
-                }
-                final int partition = tid.getPartition();
-                Map<MultipartName, Object> inputGlobalVariables = createInputGlobalVariables(ctxVarMap, han);
-                Task task = new Task(joblet, taId, han.getClass().getName(), executor);
-                IOperatorEnvironment env = joblet.getEnvironment(tid.getActivityId().getOperatorDescriptorId(),
-                        tid.getPartition());
-                IOperatorNodePushable operator = han.createPushRuntime(task, env, rdp, partition,
-                        td.getPartitionCount());
-
-                List<IPartitionCollector> collectors = new ArrayList<IPartitionCollector>();
-
-                List<IConnectorDescriptor> inputs = plan.getActivityInputConnectorDescriptors(tid.getActivityId());
-                if (inputs != null) {
-                    for (int i = 0; i < inputs.size(); ++i) {
-                        IConnectorDescriptor conn = inputs.get(i);
-                        IConnectorPolicy cPolicy = connectorPoliciesMap.get(conn.getConnectorId());
-                        if (LOGGER.isLoggable(Level.INFO)) {
-                            LOGGER.info("input: " + i + ": " + conn.getConnectorId());
-                        }
-                        RecordDescriptor recordDesc = plan.getJobSpecification().getConnectorRecordDescriptor(conn);
-                        IPartitionCollector collector = createPartitionCollector(td, partition, task, i, conn,
-                                recordDesc, cPolicy);
-                        collectors.add(collector);
-                    }
-                }
-                List<IConnectorDescriptor> outputs = plan.getActivityOutputConnectorDescriptors(tid.getActivityId());
-                if (outputs != null) {
-                    for (int i = 0; i < outputs.size(); ++i) {
-                        final IConnectorDescriptor conn = outputs.get(i);
-                        RecordDescriptor recordDesc = plan.getJobSpecification().getConnectorRecordDescriptor(conn);
-                        IConnectorPolicy cPolicy = connectorPoliciesMap.get(conn.getConnectorId());
-
-                        IPartitionWriterFactory pwFactory = createPartitionWriterFactory(cPolicy, jobId, conn,
-                                partition, taId);
-
-                        if (LOGGER.isLoggable(Level.INFO)) {
-                            LOGGER.info("output: " + i + ": " + conn.getConnectorId());
-                        }
-                        IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition,
-                                td.getPartitionCount(), td.getOutputPartitionCounts()[i]);
-                        operator.setOutputFrameWriter(i, writer, recordDesc);
-                    }
-                }
-
-                task.setTaskRuntime(collectors.toArray(new IPartitionCollector[collectors.size()]), operator);
-                joblet.addTask(task);
-
-                task.start();
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw e;
-        }
-    }
-
-    private Map<MultipartName, Object> createInputGlobalVariables(Map<MultipartName, Object> ctxVarMap, IActivity han) {
-        Map<MultipartName, Object> gVars = new HashMap<MultipartName, Object>();
-        //        for (MultipartName inVar : han.getInputVariables()) {
-        //            gVars.put(inVar, ctxVarMap.get(inVar));
-        //        }
-        return gVars;
-    }
-
-    private IPartitionCollector createPartitionCollector(TaskAttemptDescriptor td, final int partition, Task task,
-            int i, IConnectorDescriptor conn, RecordDescriptor recordDesc, IConnectorPolicy cPolicy)
-            throws HyracksDataException {
-        IPartitionCollector collector = conn.createPartitionCollector(task, recordDesc, partition,
-                td.getInputPartitionCounts()[i], td.getPartitionCount());
-        if (cPolicy.materializeOnReceiveSide()) {
-            return new ReceiveSideMaterializingCollector(ctx, partitionManager, collector, task.getTaskAttemptId(),
-                    executor);
-        } else {
-            return collector;
-        }
-    }
-
-    private IPartitionWriterFactory createPartitionWriterFactory(IConnectorPolicy cPolicy, final JobId jobId,
-            final IConnectorDescriptor conn, final int senderIndex, final TaskAttemptId taId) {
-        if (cPolicy.materializeOnSendSide()) {
-            return new IPartitionWriterFactory() {
-                @Override
-                public IFrameWriter createFrameWriter(int receiverIndex) throws HyracksDataException {
-                    return new MaterializedPartitionWriter(ctx, partitionManager, new PartitionId(jobId,
-                            conn.getConnectorId(), senderIndex, receiverIndex), taId, executor);
-                }
-            };
-        } else {
-            return new IPartitionWriterFactory() {
-                @Override
-                public IFrameWriter createFrameWriter(int receiverIndex) throws HyracksDataException {
-                    return new PipelinedPartition(partitionManager, new PartitionId(jobId, conn.getConnectorId(),
-                            senderIndex, receiverIndex), taId);
-                }
-            };
-        }
-    }
-
-    private synchronized Joblet getOrCreateLocalJoblet(JobId jobId, INCApplicationContext appCtx) throws Exception {
-        Joblet ji = jobletMap.get(jobId);
-        if (ji == null) {
-            ji = new Joblet(this, jobId, appCtx);
-            jobletMap.put(jobId, ji);
-        }
-        return ji;
-    }
-
-    public Executor getExecutor() {
-        return executor;
-    }
-
-    @Override
-    public void cleanUpJob(JobId jobId) throws Exception {
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Cleaning up after job: " + jobId);
-        }
-        Joblet joblet = jobletMap.remove(jobId);
-        if (joblet != null) {
-            partitionManager.unregisterPartitions(jobId);
-            joblet.close();
-        }
-    }
-
-    public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, TaskProfile taskProfile) throws Exception {
-        try {
-            ccs.notifyTaskComplete(jobId, taskId, id, taskProfile);
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw e;
-        }
-    }
-
-    public void notifyTaskFailed(JobId jobId, TaskAttemptId taskId, Exception exception) {
-        try {
-            ccs.notifyTaskFailure(jobId, taskId, id, exception);
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-
-    @Override
-    public void notifyRegistration(IClusterController ccs) throws Exception {
-        this.ccs = ccs;
-    }
-
-    @Override
-    public NCConfig getConfiguration() throws Exception {
-        return ncConfig;
-    }
-
     private class HeartbeatTask extends TimerTask {
         private IClusterController cc;
 
@@ -457,24 +325,10 @@
         @Override
         public void run() {
             try {
-                List<JobProfile> profiles;
-                synchronized (NodeControllerService.this) {
-                    profiles = new ArrayList<JobProfile>();
-                    for (Joblet ji : jobletMap.values()) {
-                        profiles.add(new JobProfile(ji.getJobId()));
-                    }
-                }
-                for (JobProfile jProfile : profiles) {
-                    Joblet ji;
-                    JobletProfile jobletProfile = new JobletProfile(id);
-                    synchronized (NodeControllerService.this) {
-                        ji = jobletMap.get(jProfile.getJobId());
-                    }
-                    if (ji != null) {
-                        ji.dumpProfile(jobletProfile);
-                        jProfile.getJobletProfiles().put(id, jobletProfile);
-                    }
-                }
+                FutureValue fv = new FutureValue();
+                BuildJobProfilesWork bjpw = new BuildJobProfilesWork(NodeControllerService.this, fv);
+                queue.scheduleAndSync(bjpw);
+                List<JobProfile> profiles = (List<JobProfile>) fv.get();
                 if (!profiles.isEmpty()) {
                     cc.reportProfile(id, profiles);
                 }
@@ -483,70 +337,4 @@
             }
         }
     }
-
-    @Override
-    public synchronized void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception {
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Aborting Tasks: " + jobId + ":" + tasks);
-        }
-        Joblet ji = jobletMap.get(jobId);
-        if (ji != null) {
-            Map<TaskAttemptId, Task> taskMap = ji.getTaskMap();
-            for (TaskAttemptId taId : tasks) {
-                Task task = taskMap.get(taId);
-                if (task != null) {
-                    task.abort();
-                }
-            }
-            ji.close();
-        }
-    }
-
-    @Override
-    public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
-            throws Exception {
-        NCApplicationContext appCtx;
-        synchronized (applications) {
-            if (applications.containsKey(appName)) {
-                throw new HyracksException("Duplicate application with name: " + appName + " being created.");
-            }
-            appCtx = new NCApplicationContext(serverCtx, ctx, appName, id);
-            applications.put(appName, appCtx);
-        }
-        if (deployHar) {
-            HttpClient hc = new DefaultHttpClient();
-            HttpGet get = new HttpGet("http://" + ncConfig.ccHost + ":"
-                    + nodeParameters.getClusterControllerInfo().getWebPort() + "/applications/" + appName);
-            HttpResponse response = hc.execute(get);
-            InputStream is = response.getEntity().getContent();
-            OutputStream os = appCtx.getHarOutputStream();
-            try {
-                IOUtils.copyLarge(is, os);
-            } finally {
-                os.close();
-                is.close();
-            }
-        }
-        appCtx.initializeClassPath();
-        appCtx.setDistributedState((Serializable) appCtx.deserialize(serializedDistributedState));
-        appCtx.initialize();
-    }
-
-    @Override
-    public void destroyApplication(String appName) throws Exception {
-        ApplicationContext appCtx = applications.remove(appName);
-        if (appCtx != null) {
-            appCtx.deinitialize();
-        }
-    }
-
-    @Override
-    public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception {
-        Joblet ji = jobletMap.get(pid.getJobId());
-        if (ji != null) {
-            PartitionChannel channel = new PartitionChannel(pid, new NetworkInputChannel(ctx, connectionManager,
-                    new InetSocketAddress(networkAddress.getIpAddress(), networkAddress.getPort()), pid, 1));
-            ji.reportPartitionAvailability(channel);
-        }
-    }
 }
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index 6a73902..e6529d7 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -15,7 +15,6 @@
 package edu.uci.ics.hyracks.control.nc;
 
 import java.nio.ByteBuffer;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.Executor;
@@ -26,7 +25,6 @@
 import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
 import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -36,9 +34,7 @@
 import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
 import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.naming.MultipartName;
 import edu.uci.ics.hyracks.api.resources.IDeallocatable;
-import edu.uci.ics.hyracks.api.workflow.variables.WorkflowVariableDescriptor;
 import edu.uci.ics.hyracks.control.common.job.PartitionState;
 import edu.uci.ics.hyracks.control.common.job.profiling.counters.Counter;
 import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
@@ -61,12 +57,6 @@
 
     private final Map<String, Counter> counterMap;
 
-    private final Map<MultipartName, Object> inputGlobalVariables;
-
-    private final Map<MultipartName, Object> outputVariables;
-
-    private final Map<MultipartName, WorkflowVariableDescriptor> outputVariableDescriptorMap;
-
     private IPartitionCollector[] collectors;
 
     private IOperatorNodePushable operator;
@@ -81,13 +71,6 @@
         fileFactory = new WorkspaceFileFactory(this, (IOManager) joblet.getIOManager());
         deallocatableRegistry = new DefaultDeallocatableRegistry();
         counterMap = new HashMap<String, Counter>();
-        //        this.inputGlobalVariables = inputGlobalVariables;
-        inputGlobalVariables = Collections.emptyMap();
-        outputVariables = new HashMap<MultipartName, Object>();
-        outputVariableDescriptorMap = new HashMap<MultipartName, WorkflowVariableDescriptor>();
-        //        for (WorkflowVariableDescriptor wvd : outputVariableDescriptors) {
-        //            outputVariableDescriptorMap.put(wvd.getName(), wvd);
-        //        }
     }
 
     public void setTaskRuntime(IPartitionCollector[] collectors, IOperatorNodePushable operator) {
@@ -125,28 +108,6 @@
         deallocatableRegistry.registerDeallocatable(deallocatable);
     }
 
-    @Override
-    public Object lookupGlobalVariable(ActivityId producerActivity, int partition, String varName)
-            throws HyracksDataException {
-        MultipartName var = new MultipartName(producerActivity, partition, varName);
-        if (!inputGlobalVariables.containsKey(var)) {
-            throw new HyracksDataException("Unknown Variable: " + var);
-        }
-        return inputGlobalVariables.get(var);
-    }
-
-    @Override
-    public Object lookupLocalVariable(ActivityId producerActivity, int partition, String varName)
-            throws HyracksDataException {
-        return joblet.lookupLocalVariable(new MultipartName(producerActivity, partition, varName));
-    }
-
-    @Override
-    public void setVariable(String name, Object value) {
-        outputVariables.put(new MultipartName(taskAttemptId.getTaskId().getActivityId(), taskAttemptId.getTaskId()
-                .getPartition(), name), value);
-    }
-
     public void close() {
         deallocatableRegistry.close();
     }
@@ -229,27 +190,14 @@
             } finally {
                 operator.deinitialize();
             }
-            Map<MultipartName, Object> outputGlobalVariables = new HashMap<MultipartName, Object>();
-            for (Map.Entry<MultipartName, Object> e : outputVariables.entrySet()) {
-                MultipartName varName = e.getKey();
-                WorkflowVariableDescriptor wvd = outputVariableDescriptorMap.get(varName);
-                if (wvd == null) {
-                    throw new HyracksDataException("Unknown variable found: " + varName);
-                }
-                switch (wvd.getScope()) {
-                    case LOCAL:
-                        joblet.setLocalVariable(varName, e.getValue());
-                        break;
-
-                    case GLOBAL:
-                        outputGlobalVariables.put(varName, e.getValue());
-                        break;
-                }
-            }
             joblet.notifyTaskComplete(this);
         } catch (Exception e) {
             e.printStackTrace();
-            joblet.notifyTaskFailed(this, e);
+            try {
+                joblet.notifyTaskFailed(this, e);
+            } catch (Exception e1) {
+                e1.printStackTrace();
+            }
         } finally {
             ct.setName(threadName);
             close();
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
index 06f4212..1121c6c 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/application/NCApplicationContext.java
@@ -26,7 +26,6 @@
         return nodeId;
     }
 
-    @Override
     public void setDistributedState(Serializable state) {
         distributedState = state;
     }
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java
index df76604..f93f874 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java
@@ -136,7 +136,7 @@
                         try {
                             wait();
                         } catch (InterruptedException e) {
-                            e.printStackTrace();
+                            throw new HyracksDataException(e);
                         }
                     }
                 }
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/AbortTasksWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/AbortTasksWork.java
new file mode 100644
index 0000000..6549a74
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/AbortTasksWork.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.work;
+
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+import edu.uci.ics.hyracks.control.nc.Joblet;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.Task;
+
+public class AbortTasksWork extends SynchronizableWork {
+    private static final Logger LOGGER = Logger.getLogger(AbortTasksWork.class.getName());
+
+    private final NodeControllerService ncs;
+
+    private final JobId jobId;
+
+    private final List<TaskAttemptId> tasks;
+
+    public AbortTasksWork(NodeControllerService ncs, JobId jobId, List<TaskAttemptId> tasks) {
+        this.ncs = ncs;
+        this.jobId = jobId;
+        this.tasks = tasks;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Aborting Tasks: " + jobId + ":" + tasks);
+        }
+        Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
+        Joblet ji = jobletMap.get(jobId);
+        if (ji != null) {
+            Map<TaskAttemptId, Task> taskMap = ji.getTaskMap();
+            for (TaskAttemptId taId : tasks) {
+                Task task = taskMap.get(taId);
+                if (task != null) {
+                    task.abort();
+                }
+            }
+            ji.close();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/BuildJobProfilesWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/BuildJobProfilesWork.java
new file mode 100644
index 0000000..0aee348
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/BuildJobProfilesWork.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.work;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobletProfile;
+import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+import edu.uci.ics.hyracks.control.nc.Joblet;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+
+public class BuildJobProfilesWork extends SynchronizableWork {
+    private static final Logger LOGGER = Logger.getLogger(BuildJobProfilesWork.class.getName());
+
+    private final NodeControllerService ncs;
+
+    private final FutureValue fv;
+
+    public BuildJobProfilesWork(NodeControllerService ncs, FutureValue fv) {
+        this.ncs = ncs;
+        this.fv = fv;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        List<JobProfile> profiles = new ArrayList<JobProfile>();
+        Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
+        for (Joblet ji : jobletMap.values()) {
+            profiles.add(new JobProfile(ji.getJobId()));
+        }
+        for (JobProfile jProfile : profiles) {
+            Joblet ji;
+            JobletProfile jobletProfile = new JobletProfile(ncs.getId());
+            ji = jobletMap.get(jProfile.getJobId());
+            if (ji != null) {
+                ji.dumpProfile(jobletProfile);
+                jProfile.getJobletProfiles().put(ncs.getId(), jobletProfile);
+            }
+        }
+        fv.setValue(profiles);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CleanupJobWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CleanupJobWork.java
new file mode 100644
index 0000000..50c05df
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CleanupJobWork.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.work;
+
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+import edu.uci.ics.hyracks.control.nc.Joblet;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+
+public class CleanupJobWork extends SynchronizableWork {
+    private static final Logger LOGGER = Logger.getLogger(CleanupJobWork.class.getName());
+
+    private final NodeControllerService ncs;
+
+    private final JobId jobId;
+
+    public CleanupJobWork(NodeControllerService ncs, JobId jobId) {
+        this.ncs = ncs;
+        this.jobId = jobId;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Cleaning up after job: " + jobId);
+        }
+        Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
+        Joblet joblet = jobletMap.remove(jobId);
+        if (joblet != null) {
+            ncs.getPartitionManager().unregisterPartitions(jobId);
+            joblet.close();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
new file mode 100644
index 0000000..120e376
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/CreateApplicationWork.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.work;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.Map;
+import java.util.logging.Logger;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.impl.client.DefaultHttpClient;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
+
+public class CreateApplicationWork extends SynchronizableWork {
+    private static final Logger LOGGER = Logger.getLogger(CreateApplicationWork.class.getName());
+
+    private final NodeControllerService ncs;
+
+    private final String appName;
+
+    private final boolean deployHar;
+
+    private final byte[] serializedDistributedState;
+
+    public CreateApplicationWork(NodeControllerService ncs, String appName, boolean deployHar,
+            byte[] serializedDistributedState) {
+        this.ncs = ncs;
+        this.appName = appName;
+        this.deployHar = deployHar;
+        this.serializedDistributedState = serializedDistributedState;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        NCApplicationContext appCtx;
+        Map<String, NCApplicationContext> applications = ncs.getApplications();
+        if (applications.containsKey(appName)) {
+            throw new HyracksException("Duplicate application with name: " + appName + " being created.");
+        }
+        appCtx = new NCApplicationContext(ncs.getServerContext(), ncs.getRootContext(), appName, ncs.getId());
+        applications.put(appName, appCtx);
+        if (deployHar) {
+            NCConfig ncConfig = ncs.getConfiguration();
+            NodeParameters nodeParameters = ncs.getNodeParameters();
+            HttpClient hc = new DefaultHttpClient();
+            HttpGet get = new HttpGet("http://" + ncConfig.ccHost + ":"
+                    + nodeParameters.getClusterControllerInfo().getWebPort() + "/applications/" + appName);
+            HttpResponse response = hc.execute(get);
+            InputStream is = response.getEntity().getContent();
+            OutputStream os = appCtx.getHarOutputStream();
+            try {
+                IOUtils.copyLarge(is, os);
+            } finally {
+                os.close();
+                is.close();
+            }
+        }
+        appCtx.initializeClassPath();
+        appCtx.setDistributedState((Serializable) appCtx.deserialize(serializedDistributedState));
+        appCtx.initialize();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
new file mode 100644
index 0000000..6a5fbfc
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/DestroyApplicationWork.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.work;
+
+import java.util.Map;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
+
+public class DestroyApplicationWork extends SynchronizableWork {
+    private static final Logger LOGGER = Logger.getLogger(DestroyApplicationWork.class.getName());
+
+    private final NodeControllerService ncs;
+
+    private final String appName;
+
+    public DestroyApplicationWork(NodeControllerService ncs, String appName) {
+        this.ncs = ncs;
+        this.appName = appName;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        Map<String, NCApplicationContext> applications = ncs.getApplications();
+        ApplicationContext appCtx = applications.remove(appName);
+        if (appCtx != null) {
+            appCtx.deinitialize();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
new file mode 100644
index 0000000..bfa21c9
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.work;
+
+import java.net.InetSocketAddress;
+import java.util.Map;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+import edu.uci.ics.hyracks.control.nc.Joblet;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.net.NetworkInputChannel;
+
+public class ReportPartitionAvailabilityWork extends SynchronizableWork {
+    private static final Logger LOGGER = Logger.getLogger(ReportPartitionAvailabilityWork.class.getName());
+
+    private final NodeControllerService ncs;
+
+    private final PartitionId pid;
+
+    private final NetworkAddress networkAddress;
+
+    public ReportPartitionAvailabilityWork(NodeControllerService ncs, PartitionId pid, NetworkAddress networkAddress) {
+        this.ncs = ncs;
+        this.pid = pid;
+        this.networkAddress = networkAddress;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
+        Joblet ji = jobletMap.get(pid.getJobId());
+        if (ji != null) {
+            PartitionChannel channel = new PartitionChannel(pid, new NetworkInputChannel(ncs.getRootContext(),
+                    ncs.getConnectionManager(), new InetSocketAddress(networkAddress.getIpAddress(),
+                            networkAddress.getPort()), pid, 1));
+            ji.reportPartitionAvailability(channel);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
new file mode 100644
index 0000000..ed83451
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
@@ -0,0 +1,203 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.work;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+import edu.uci.ics.hyracks.control.nc.Joblet;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.Task;
+import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
+import edu.uci.ics.hyracks.control.nc.partitions.MaterializedPartitionWriter;
+import edu.uci.ics.hyracks.control.nc.partitions.PipelinedPartition;
+import edu.uci.ics.hyracks.control.nc.partitions.ReceiveSideMaterializingCollector;
+
+public class StartTasksWork extends SynchronizableWork {
+    private static final Logger LOGGER = Logger.getLogger(StartTasksWork.class.getName());
+
+    private final NodeControllerService ncs;
+
+    private final String appName;
+
+    private final JobId jobId;
+
+    private final byte[] jagBytes;
+
+    private final List<TaskAttemptDescriptor> taskDescriptors;
+
+    private final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap;
+
+    public StartTasksWork(NodeControllerService ncs, String appName, JobId jobId, byte[] jagBytes,
+            List<TaskAttemptDescriptor> taskDescriptors,
+            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap) {
+        this.ncs = ncs;
+        this.appName = appName;
+        this.jobId = jobId;
+        this.jagBytes = jagBytes;
+        this.taskDescriptors = taskDescriptors;
+        this.connectorPoliciesMap = connectorPoliciesMap;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        try {
+            Map<String, NCApplicationContext> applications = ncs.getApplications();
+            NCApplicationContext appCtx = applications.get(appName);
+            final JobActivityGraph plan = (JobActivityGraph) appCtx.deserialize(jagBytes);
+
+            IRecordDescriptorProvider rdp = new IRecordDescriptorProvider() {
+                @Override
+                public RecordDescriptor getOutputRecordDescriptor(OperatorDescriptorId opId, int outputIndex) {
+                    return plan.getJobSpecification().getOperatorOutputRecordDescriptor(opId, outputIndex);
+                }
+
+                @Override
+                public RecordDescriptor getInputRecordDescriptor(OperatorDescriptorId opId, int inputIndex) {
+                    return plan.getJobSpecification().getOperatorInputRecordDescriptor(opId, inputIndex);
+                }
+            };
+
+            final Joblet joblet = getOrCreateLocalJoblet(jobId, appCtx);
+
+            for (TaskAttemptDescriptor td : taskDescriptors) {
+                TaskAttemptId taId = td.getTaskAttemptId();
+                TaskId tid = taId.getTaskId();
+                IActivity han = plan.getActivityNodeMap().get(tid.getActivityId());
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Initializing " + taId + " -> " + han);
+                }
+                final int partition = tid.getPartition();
+                Task task = new Task(joblet, taId, han.getClass().getName(), ncs.getExecutor());
+                IOperatorEnvironment env = joblet.getEnvironment(tid.getActivityId().getOperatorDescriptorId(),
+                        tid.getPartition());
+                IOperatorNodePushable operator = han.createPushRuntime(task, env, rdp, partition,
+                        td.getPartitionCount());
+
+                List<IPartitionCollector> collectors = new ArrayList<IPartitionCollector>();
+
+                List<IConnectorDescriptor> inputs = plan.getActivityInputConnectorDescriptors(tid.getActivityId());
+                if (inputs != null) {
+                    for (int i = 0; i < inputs.size(); ++i) {
+                        IConnectorDescriptor conn = inputs.get(i);
+                        IConnectorPolicy cPolicy = connectorPoliciesMap.get(conn.getConnectorId());
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info("input: " + i + ": " + conn.getConnectorId());
+                        }
+                        RecordDescriptor recordDesc = plan.getJobSpecification().getConnectorRecordDescriptor(conn);
+                        IPartitionCollector collector = createPartitionCollector(td, partition, task, i, conn,
+                                recordDesc, cPolicy);
+                        collectors.add(collector);
+                    }
+                }
+                List<IConnectorDescriptor> outputs = plan.getActivityOutputConnectorDescriptors(tid.getActivityId());
+                if (outputs != null) {
+                    for (int i = 0; i < outputs.size(); ++i) {
+                        final IConnectorDescriptor conn = outputs.get(i);
+                        RecordDescriptor recordDesc = plan.getJobSpecification().getConnectorRecordDescriptor(conn);
+                        IConnectorPolicy cPolicy = connectorPoliciesMap.get(conn.getConnectorId());
+
+                        IPartitionWriterFactory pwFactory = createPartitionWriterFactory(cPolicy, jobId, conn,
+                                partition, taId);
+
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info("output: " + i + ": " + conn.getConnectorId());
+                        }
+                        IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition,
+                                td.getPartitionCount(), td.getOutputPartitionCounts()[i]);
+                        operator.setOutputFrameWriter(i, writer, recordDesc);
+                    }
+                }
+
+                task.setTaskRuntime(collectors.toArray(new IPartitionCollector[collectors.size()]), operator);
+                joblet.addTask(task);
+
+                task.start();
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw e;
+        }
+    }
+
+    private Joblet getOrCreateLocalJoblet(JobId jobId, INCApplicationContext appCtx) throws Exception {
+        Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
+        Joblet ji = jobletMap.get(jobId);
+        if (ji == null) {
+            ji = new Joblet(ncs, jobId, appCtx);
+            jobletMap.put(jobId, ji);
+        }
+        return ji;
+    }
+
+    private IPartitionCollector createPartitionCollector(TaskAttemptDescriptor td, final int partition, Task task,
+            int i, IConnectorDescriptor conn, RecordDescriptor recordDesc, IConnectorPolicy cPolicy)
+            throws HyracksDataException {
+        IPartitionCollector collector = conn.createPartitionCollector(task, recordDesc, partition,
+                td.getInputPartitionCounts()[i], td.getPartitionCount());
+        if (cPolicy.materializeOnReceiveSide()) {
+            return new ReceiveSideMaterializingCollector(ncs.getRootContext(), ncs.getPartitionManager(), collector,
+                    task.getTaskAttemptId(), ncs.getExecutor());
+        } else {
+            return collector;
+        }
+    }
+
+    private IPartitionWriterFactory createPartitionWriterFactory(IConnectorPolicy cPolicy, final JobId jobId,
+            final IConnectorDescriptor conn, final int senderIndex, final TaskAttemptId taId) {
+        if (cPolicy.materializeOnSendSide()) {
+            return new IPartitionWriterFactory() {
+                @Override
+                public IFrameWriter createFrameWriter(int receiverIndex) throws HyracksDataException {
+                    return new MaterializedPartitionWriter(ncs.getRootContext(), ncs.getPartitionManager(),
+                            new PartitionId(jobId, conn.getConnectorId(), senderIndex, receiverIndex), taId,
+                            ncs.getExecutor());
+                }
+            };
+        } else {
+            return new IPartitionWriterFactory() {
+                @Override
+                public IFrameWriter createFrameWriter(int receiverIndex) throws HyracksDataException {
+                    return new PipelinedPartition(ncs.getPartitionManager(), new PartitionId(jobId,
+                            conn.getConnectorId(), senderIndex, receiverIndex), taId);
+                }
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/pom.xml b/hyracks-dataflow-common/pom.xml
index 3a0453c..50dc4d2 100644
--- a/hyracks-dataflow-common/pom.xml
+++ b/hyracks-dataflow-common/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-dataflow-common</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -27,7 +27,7 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-api</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/TupleUtils.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/TupleUtils.java
index 093d17a..1e9c47f 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/TupleUtils.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/TupleUtils.java
@@ -68,7 +68,8 @@
     public static String printTuple(ITupleReference tuple,
             ISerializerDeserializer[] fields) throws HyracksDataException {
         StringBuilder strBuilder = new StringBuilder();
-        for (int i = 0; i < fields.length; i++) {
+        int numPrintFields = Math.min(tuple.getFieldCount(), fields.length);
+        for (int i = 0; i < numPrintFields; i++) {
             ByteArrayInputStream inStream = new ByteArrayInputStream(
                     tuple.getFieldData(i), tuple.getFieldStart(i),
                     tuple.getFieldLength(i));
diff --git a/hyracks-dataflow-hadoop/pom.xml b/hyracks-dataflow-hadoop/pom.xml
index c8ef50e..e3f9b1e 100644
--- a/hyracks-dataflow-hadoop/pom.xml
+++ b/hyracks-dataflow-hadoop/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-dataflow-hadoop</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -27,14 +27,14 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-api</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
@@ -54,7 +54,7 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   </dependencies>
diff --git a/hyracks-dataflow-std/pom.xml b/hyracks-dataflow-std/pom.xml
index 79ce38c..4e9b42b 100644
--- a/hyracks-dataflow-std/pom.xml
+++ b/hyracks-dataflow-std/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-dataflow-std</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -27,14 +27,14 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-api</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
index 647d750..09984aa 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
@@ -16,6 +16,7 @@
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.LinkedList;
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
@@ -48,7 +49,7 @@
             IFrameWriter writer) {
         this.ctx = ctx;
         this.frameSorter = frameSorter;
-        this.runs = runs;
+        this.runs = new LinkedList<IFrameReader>(runs);
         this.sortFields = sortFields;
         this.comparators = comparators;
         this.recordDesc = recordDesc;
diff --git a/hyracks-documentation/pom.xml b/hyracks-documentation/pom.xml
index 522011a..47d997b 100644
--- a/hyracks-documentation/pom.xml
+++ b/hyracks-documentation/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-documentation</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
diff --git a/hyracks-examples/btree-example/btreeapp/pom.xml b/hyracks-examples/btree-example/btreeapp/pom.xml
index 737eaa0..d5239bd 100644
--- a/hyracks-examples/btree-example/btreeapp/pom.xml
+++ b/hyracks-examples/btree-example/btreeapp/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.btree</groupId>
   <artifactId>btreeapp</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>btree-example</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -80,7 +80,7 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks.examples.btree</groupId>
   		<artifactId>btreehelper</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   </dependencies>
diff --git a/hyracks-examples/btree-example/btreeclient/pom.xml b/hyracks-examples/btree-example/btreeclient/pom.xml
index dea042a..a95d17e 100644
--- a/hyracks-examples/btree-example/btreeclient/pom.xml
+++ b/hyracks-examples/btree-example/btreeclient/pom.xml
@@ -2,31 +2,31 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.btree</groupId>
   <artifactId>btreeclient</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>btree-example</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <dependencies>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-am-btree</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks.examples.btree</groupId>
   		<artifactId>btreehelper</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
diff --git a/hyracks-examples/btree-example/btreehelper/pom.xml b/hyracks-examples/btree-example/btreehelper/pom.xml
index f790b5a..2615dc2 100644
--- a/hyracks-examples/btree-example/btreehelper/pom.xml
+++ b/hyracks-examples/btree-example/btreehelper/pom.xml
@@ -2,31 +2,31 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.btree</groupId>
   <artifactId>btreehelper</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>btree-example</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <dependencies>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-am-btree</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-api</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   </dependencies>
diff --git a/hyracks-examples/btree-example/pom.xml b/hyracks-examples/btree-example/pom.xml
index de4258d..96b3672 100644
--- a/hyracks-examples/btree-example/pom.xml
+++ b/hyracks-examples/btree-example/pom.xml
@@ -2,13 +2,13 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples</groupId>
   <artifactId>btree-example</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks-examples</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <modules>
diff --git a/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml b/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
index 5fc56a1..15d31b6 100644
--- a/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
+++ b/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.compat</groupId>
   <artifactId>hadoopcompatapp</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>hadoop-compat-example</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -169,13 +169,13 @@
      <dependency>
         <groupId>edu.uci.ics.hyracks.examples.compat</groupId>
         <artifactId>hadoopcompathelper</artifactId>
-        <version>0.1.8-SNAPSHOT</version>
+        <version>0.2.0-SNAPSHOT</version>
         <scope>compile</scope>
      </dependency>
      <dependency>
         <groupId>edu.uci.ics.hyracks.examples.compat</groupId>
   	    <artifactId>hadoopcompatclient</artifactId>
-  	    <version>0.1.8-SNAPSHOT</version>
+  	    <version>0.2.0-SNAPSHOT</version>
   	    <type>jar</type>
   	    <scope>test</scope>
      </dependency>
diff --git a/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml b/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
index 5c343de..c248d55 100644
--- a/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
+++ b/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
@@ -2,25 +2,25 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.compat</groupId>
   <artifactId>hadoopcompatclient</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>hadoop-compat-example</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <dependencies>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks.examples.compat</groupId>
   		<artifactId>hadoopcompathelper</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
diff --git a/hyracks-examples/hadoop-compat-example/hadoopcompathelper/pom.xml b/hyracks-examples/hadoop-compat-example/hadoopcompathelper/pom.xml
index b45f0e9..c19e56a 100644
--- a/hyracks-examples/hadoop-compat-example/hadoopcompathelper/pom.xml
+++ b/hyracks-examples/hadoop-compat-example/hadoopcompathelper/pom.xml
@@ -2,25 +2,25 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.compat</groupId>
   <artifactId>hadoopcompathelper</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>hadoop-compat-example</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <dependencies>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-api</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   </dependencies>
diff --git a/hyracks-examples/hadoop-compat-example/pom.xml b/hyracks-examples/hadoop-compat-example/pom.xml
index 6310d29..51e92ab 100644
--- a/hyracks-examples/hadoop-compat-example/pom.xml
+++ b/hyracks-examples/hadoop-compat-example/pom.xml
@@ -2,13 +2,13 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples</groupId>
   <artifactId>hadoop-compat-example</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks-examples</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <modules>
@@ -29,7 +29,7 @@
       <dependency>
          <groupId>edu.uci.ics.hyracks</groupId>
          <artifactId>hyracks-hadoop-compat</artifactId>
-         <version>0.1.8-SNAPSHOT</version>
+         <version>0.2.0-SNAPSHOT</version>
          <type>jar</type>
          <scope>compile</scope>
       </dependency>
diff --git a/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks-examples/hyracks-integration-tests/pom.xml
index 2006dc9..81507fc 100644
--- a/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples</groupId>
   <artifactId>hyracks-integration-tests</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks-examples</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -42,49 +42,49 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-cc</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-nc</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-am-btree</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-am-invertedindex</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-am-rtree</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-test-support</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>test</scope>
   	</dependency>
diff --git a/hyracks-examples/pom.xml b/hyracks-examples/pom.xml
index 7e66a6e..a7021a9 100644
--- a/hyracks-examples/pom.xml
+++ b/hyracks-examples/pom.xml
@@ -2,13 +2,13 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-examples</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <modules>
diff --git a/hyracks-examples/text-example/pom.xml b/hyracks-examples/text-example/pom.xml
index d18f0cb..d334644 100644
--- a/hyracks-examples/text-example/pom.xml
+++ b/hyracks-examples/text-example/pom.xml
@@ -2,13 +2,13 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples</groupId>
   <artifactId>text-example</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks-examples</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <modules>
diff --git a/hyracks-examples/text-example/textapp/pom.xml b/hyracks-examples/text-example/textapp/pom.xml
index dd1b5ae..3b3b9fd 100644
--- a/hyracks-examples/text-example/textapp/pom.xml
+++ b/hyracks-examples/text-example/textapp/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.text</groupId>
   <artifactId>textapp</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>text-example</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -164,13 +164,13 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks.examples.text</groupId>
   		<artifactId>texthelper</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks.examples.text</groupId>
   		<artifactId>textclient</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>test</scope>
   	</dependency>
diff --git a/hyracks-examples/text-example/textclient/pom.xml b/hyracks-examples/text-example/textclient/pom.xml
index 42e9655..2642d31 100644
--- a/hyracks-examples/text-example/textclient/pom.xml
+++ b/hyracks-examples/text-example/textclient/pom.xml
@@ -2,25 +2,25 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.text</groupId>
   <artifactId>textclient</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>text-example</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <dependencies>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks.examples.text</groupId>
   		<artifactId>texthelper</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
diff --git a/hyracks-examples/text-example/texthelper/pom.xml b/hyracks-examples/text-example/texthelper/pom.xml
index c21b5a3..60c423d 100644
--- a/hyracks-examples/text-example/texthelper/pom.xml
+++ b/hyracks-examples/text-example/texthelper/pom.xml
@@ -2,25 +2,25 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.text</groupId>
   <artifactId>texthelper</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>text-example</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <dependencies>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-api</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   </dependencies>
diff --git a/hyracks-examples/tpch-example/pom.xml b/hyracks-examples/tpch-example/pom.xml
index 60b00f1..f2c8786 100644
--- a/hyracks-examples/tpch-example/pom.xml
+++ b/hyracks-examples/tpch-example/pom.xml
@@ -2,13 +2,13 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples</groupId>
   <artifactId>tpch-example</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks-examples</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <modules>
diff --git a/hyracks-examples/tpch-example/tpchapp/pom.xml b/hyracks-examples/tpch-example/tpchapp/pom.xml
index 3ea1e99..c65aaf5 100644
--- a/hyracks-examples/tpch-example/tpchapp/pom.xml
+++ b/hyracks-examples/tpch-example/tpchapp/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.tpch</groupId>
   <artifactId>tpchapp</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>tpch-example</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -80,7 +80,7 @@
     <dependency>
         <groupId>edu.uci.ics.hyracks</groupId>
         <artifactId>hyracks-dataflow-std</artifactId>
-        <version>0.1.8-SNAPSHOT</version>
+        <version>0.2.0-SNAPSHOT</version>
         <scope>compile</scope>
     </dependency>
   </dependencies>
diff --git a/hyracks-examples/tpch-example/tpchclient/pom.xml b/hyracks-examples/tpch-example/tpchclient/pom.xml
index bc93289..c7a138a 100644
--- a/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -2,19 +2,19 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples.tpch</groupId>
   <artifactId>tpchclient</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks.examples</groupId>
     <artifactId>tpch-example</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <dependencies>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   </dependencies>
diff --git a/hyracks-hadoop-compat/pom.xml b/hyracks-hadoop-compat/pom.xml
index bb807a6..2de7e04 100644
--- a/hyracks-hadoop-compat/pom.xml
+++ b/hyracks-hadoop-compat/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-hadoop-compat</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -79,7 +79,7 @@
     <dependency>
     	<groupId>edu.uci.ics.hyracks</groupId>
     	<artifactId>hyracks-dataflow-hadoop</artifactId>
-    	<version>0.1.8-SNAPSHOT</version>
+    	<version>0.2.0-SNAPSHOT</version>
     	<type>jar</type>
     	<scope>compile</scope>
     </dependency>
diff --git a/hyracks-server/.settings/org.eclipse.jdt.core.prefs b/hyracks-server/.settings/org.eclipse.jdt.core.prefs
index b1de61e..611528f 100644
--- a/hyracks-server/.settings/org.eclipse.jdt.core.prefs
+++ b/hyracks-server/.settings/org.eclipse.jdt.core.prefs
@@ -1,6 +1,6 @@
-#Tue Oct 04 13:01:56 PDT 2011
+#Tue Sep 20 17:16:43 PDT 2011
 eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.5
-org.eclipse.jdt.core.compiler.compliance=1.5
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6
+org.eclipse.jdt.core.compiler.compliance=1.6
 org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
-org.eclipse.jdt.core.compiler.source=1.5
+org.eclipse.jdt.core.compiler.source=1.6
diff --git a/hyracks-server/pom.xml b/hyracks-server/pom.xml
index 73819b3..58325ad 100644
--- a/hyracks-server/pom.xml
+++ b/hyracks-server/pom.xml
@@ -2,17 +2,26 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-server</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
     <plugins>
       <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+        </configuration>
+      </plugin>
+      <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>appassembler-maven-plugin</artifactId>
         <executions>
@@ -27,6 +36,10 @@
                   <mainClass>edu.uci.ics.hyracks.control.nc.NCDriver</mainClass>
                   <name>hyracksnc</name>
                 </program>
+                <program>
+                  <mainClass>edu.uci.ics.hyracks.server.drivers.VirtualClusterDriver</mainClass>
+                  <name>hyracks-virtual-cluster</name>
+                </program>
               </programs>
               <repositoryLayout>flat</repositoryLayout>
               <repositoryName>lib</repositoryName>
@@ -61,21 +74,21 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-cc</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-nc</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-admin-console</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>war</type>
   		<scope>compile</scope>
   	</dependency>
diff --git a/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/drivers/VirtualClusterDriver.java b/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/drivers/VirtualClusterDriver.java
new file mode 100644
index 0000000..92a1ad5
--- /dev/null
+++ b/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/drivers/VirtualClusterDriver.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.server.drivers;
+
+import org.kohsuke.args4j.CmdLineParser;
+import org.kohsuke.args4j.Option;
+
+import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.server.process.HyracksCCProcess;
+import edu.uci.ics.hyracks.server.process.HyracksNCProcess;
+
+public class VirtualClusterDriver {
+    private static class Options {
+        @Option(name = "-n", required = false, usage = "Number of node controllers (default: 2)")
+        public int n = 2;
+
+        @Option(name = "-cc-port", required = false, usage = "CC Port (default: 1099)")
+        public int ccPort = 1099;
+    }
+
+    public static void main(String[] args) throws Exception {
+        Options options = new Options();
+        CmdLineParser cp = new CmdLineParser(options);
+        try {
+            cp.parseArgument(args);
+        } catch (Exception e) {
+            System.err.println(e.getMessage());
+            cp.printUsage(System.err);
+            return;
+        }
+
+        CCConfig ccConfig = new CCConfig();
+        ccConfig.port = options.ccPort;
+        HyracksCCProcess ccp = new HyracksCCProcess(ccConfig);
+        ccp.start();
+
+        Thread.sleep(2000);
+
+        HyracksNCProcess ncps[] = new HyracksNCProcess[options.n];
+        for (int i = 0; i < options.n; ++i) {
+            NCConfig ncConfig = new NCConfig();
+            ncConfig.ccHost = "localhost";
+            ncConfig.nodeId = "nc" + i;
+            ncConfig.dataIPAddress = "127.0.0.1";
+            ncps[i] = new HyracksNCProcess(ncConfig);
+            ncps[i].start();
+        }
+
+        while (true) {
+            Thread.sleep(10000);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/process/HyracksCCProcess.java b/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/process/HyracksCCProcess.java
new file mode 100644
index 0000000..cb1be95
--- /dev/null
+++ b/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/process/HyracksCCProcess.java
@@ -0,0 +1,24 @@
+package edu.uci.ics.hyracks.server.process;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.control.cc.CCDriver;
+import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
+
+public class HyracksCCProcess extends HyracksServerProcess {
+    private CCConfig config;
+
+    public HyracksCCProcess(CCConfig config) {
+        this.config = config;
+    }
+
+    @Override
+    protected void addCmdLineArgs(List<String> cList) {
+        config.toCommandLine(cList);
+    }
+
+    @Override
+    protected String getMainClassName() {
+        return CCDriver.class.getName();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/process/HyracksNCProcess.java b/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/process/HyracksNCProcess.java
new file mode 100644
index 0000000..cd110c8
--- /dev/null
+++ b/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/process/HyracksNCProcess.java
@@ -0,0 +1,24 @@
+package edu.uci.ics.hyracks.server.process;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.control.nc.NCDriver;
+
+public class HyracksNCProcess extends HyracksServerProcess {
+    private NCConfig config;
+
+    public HyracksNCProcess(NCConfig config) {
+        this.config = config;
+    }
+
+    @Override
+    protected void addCmdLineArgs(List<String> cList) {
+        config.toCommandLine(cList);
+    }
+
+    @Override
+    protected String getMainClassName() {
+        return NCDriver.class.getName();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/process/HyracksServerProcess.java b/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/process/HyracksServerProcess.java
new file mode 100644
index 0000000..17169ac
--- /dev/null
+++ b/hyracks-server/src/main/java/edu/uci/ics/hyracks/server/process/HyracksServerProcess.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.server.process;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+public abstract class HyracksServerProcess {
+    private static final Logger LOGGER = Logger.getLogger(HyracksServerProcess.class.getName());
+
+    protected Process process;
+
+    public void start() throws IOException {
+        String[] cmd = buildCommand();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Starting command: " + Arrays.toString(cmd));
+        }
+        process = Runtime.getRuntime().exec(cmd, null, null);
+        dump(process.getInputStream());
+        dump(process.getErrorStream());
+    }
+
+    private void dump(InputStream input) {
+        final int streamBufferSize = 1000;
+        final Reader in = new InputStreamReader(input);
+        new Thread(new Runnable() {
+            public void run() {
+                try {
+                    char[] chars = new char[streamBufferSize];
+                    int c;
+                    while ((c = in.read(chars)) != -1) {
+                        if (c > 0) {
+                            System.out.print(String.valueOf(chars, 0, c));
+                        }
+                    }
+                } catch (IOException e) {
+                }
+            }
+        }).start();
+    }
+
+    private String[] buildCommand() {
+        List<String> cList = new ArrayList<String>();
+        cList.add(getJavaCommand());
+        cList.add("-classpath");
+        cList.add(getClasspath());
+        cList.add(getMainClassName());
+        addCmdLineArgs(cList);
+        return cList.toArray(new String[cList.size()]);
+    }
+
+    protected abstract void addCmdLineArgs(List<String> cList);
+
+    protected abstract String getMainClassName();
+
+    private final String getClasspath() {
+        return System.getProperty("java.class.path");
+    }
+
+    protected final String getJavaCommand() {
+        return System.getProperty("java.home") + File.separator + "bin" + File.separator + "java";
+    }
+}
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/pom.xml b/hyracks-storage-am-btree/pom.xml
index efffe5d..8ad45ac 100644
--- a/hyracks-storage-am-btree/pom.xml
+++ b/hyracks-storage-am-btree/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-storage-am-btree</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -27,35 +27,35 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  	
         <dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-am-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  	
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  	
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  	
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-nc</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  	
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
index aa2a08b..0334a7f 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
@@ -54,8 +54,8 @@
     protected static final int freeSpaceOff = tupleCountOff + 4; // 12
     protected static final int totalFreeSpaceOff = freeSpaceOff + 4; // 16
     protected static final int levelOff = totalFreeSpaceOff + 4; // 20
-    protected static final int smFlagOff = levelOff + 1; // 24
-    protected static final int uncompressedTupleCountOff = smFlagOff + 1; // 25
+    protected static final int smFlagOff = levelOff + 1; // 21
+    protected static final int uncompressedTupleCountOff = smFlagOff + 1; // 22
     protected static final int prefixTupleCountOff = uncompressedTupleCountOff + 4; // 26
 
     protected static final int prevLeafOff = prefixTupleCountOff + 4; // 30
@@ -392,9 +392,19 @@
     }
     
     @Override
-    public void printHeader() {
-        // TODO Auto-generated method stub
-
+    public String printHeader() {
+        StringBuilder strBuilder = new StringBuilder();
+        strBuilder.append("pageLsnOff:                " + pageLsnOff + "\n");
+        strBuilder.append("tupleCountOff:             " + tupleCountOff + "\n");
+        strBuilder.append("freeSpaceOff:              " + freeSpaceOff + "\n");
+        strBuilder.append("totalFreeSpaceOff:         " + totalFreeSpaceOff + "\n");
+        strBuilder.append("levelOff:                  " + levelOff + "\n");
+        strBuilder.append("smFlagOff:                 " + smFlagOff + "\n");
+        strBuilder.append("uncompressedTupleCountOff: " + uncompressedTupleCountOff + "\n");
+        strBuilder.append("prefixTupleCountOff:       " + prefixTupleCountOff + "\n");
+        strBuilder.append("prevLeafOff:               " + prevLeafOff + "\n");
+        strBuilder.append("nextLeafOff:               " + nextLeafOff + "\n");
+        return strBuilder.toString();
     }
 
     @Override
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java
index 5a2d9db..87ba3ac 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java
@@ -118,7 +118,7 @@
         if (tupleIndex == slotManager.getGreatestKeyIndicator()) {
             tupleOff = slotManager.getTupleOff(slotManager.getSlotEndOff());
             frameTuple.resetByTupleOffset(buf, tupleOff);
-            keySize = tupleWriter.bytesRequired(frameTuple, 0, tuple.getFieldCount());
+            keySize = frameTuple.getTupleSize();
             // Copy new rightmost pointer.
             System.arraycopy(buf.array(), tupleOff + keySize, buf.array(), rightLeafOff, childPtrSize);
         } else {
@@ -410,6 +410,13 @@
         frameTuple.setFieldCount(cmp.getKeyFieldCount());
     }
     
+    @Override
+    public ITreeIndexTupleReference createTupleReference() {
+        ITreeIndexTupleReference tuple = tupleWriter.createTupleReference();
+        tuple.setFieldCount(cmp.getKeyFieldCount());
+        return tuple;
+    }
+    
     // For debugging.
     public ArrayList<Integer> getChildren(MultiComparator cmp) {
         ArrayList<Integer> ret = new ArrayList<Integer>();
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java
index b2bab859..5f507f5 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java
@@ -28,7 +28,7 @@
     public int findTupleIndex(ITupleReference searchKey, ITreeIndexTupleReference frameTuple, MultiComparator multiCmp,
             FindTupleMode mode, FindTupleNoExactMatchPolicy matchPolicy) {
         if (frame.getTupleCount() <= 0) {
-            return -1;
+            return GREATEST_KEY_INDICATOR;
         }
 
         int mid;
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
index edab4c2..9ce7544 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
@@ -513,7 +513,7 @@
         int tupleIndex = ctx.leafFrame.findDeleteTupleIndex(tuple);
 
         // Will this leaf become empty?
-        if (ctx.leafFrame.getTupleCount() != 1) {
+        if (ctx.leafFrame.getTupleCount() > 1) {
             // Leaf will not become empty.
             ctx.leafFrame.delete(tuple, tupleIndex);
             node.releaseWriteLatch();
@@ -623,12 +623,10 @@
 
         } else {
             ctx.interiorFrame.delete(tuple, tupleIndex);
-            ctx.interiorFrame.setPageLsn(ctx.interiorFrame.getPageLsn() + 1); // TODO:
-            // tie
-            // together
-            // with
-            // logging
-            ctx.splitKey.reset(); // don't propagate deletion
+            // TODO: Tie together with logging.
+            ctx.interiorFrame.setPageLsn(ctx.interiorFrame.getPageLsn() + 1);
+            // Don't propagate deletion.
+            ctx.splitKey.reset();
         }
     }
 
@@ -665,6 +663,12 @@
     private void performOp(int pageId, ICachedPage parent, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
         ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
         ctx.interiorFrame.setPage(node);
+        
+        //System.out.println("PAGEID: " + pageId);
+        //System.out.println("NODE:   " + node);
+        //System.out.println("LEVEL:  " + ctx.interiorFrame.getLevel() + " " + ctx.interiorFrame.isLeaf());
+        //System.out.println("-------------------------");
+        
         // this check performs an unprotected read in the page
         // the following could happen: TODO fill out
         boolean unsafeIsLeaf = ctx.interiorFrame.isLeaf();
@@ -1088,7 +1092,7 @@
             interiorFrame.setPage(node);
             int level = interiorFrame.getLevel();
             strBuilder.append(String.format("%1d ", level));
-            strBuilder.append(String.format("%3d ", pageId));
+            strBuilder.append(String.format("%3d ", pageId) + ": ");
             for (int i = 0; i < treeHeight - level; i++) {
                 strBuilder.append("    ");
             }
@@ -1101,7 +1105,7 @@
                 keyString = TreeIndexUtils.printFrameTuples(interiorFrame, keySerdes);
             }
 
-            strBuilder.append(keyString);
+            strBuilder.append(keyString + "\n");
             if (!interiorFrame.isLeaf()) {
                 ArrayList<Integer> children = ((BTreeNSMInteriorFrame) (interiorFrame)).getChildren(cmp);
                 for (int i = 0; i < children.size(); i++) {
diff --git a/hyracks-storage-am-common/pom.xml b/hyracks-storage-am-common/pom.xml
index 783faed..9f0e53a 100644
--- a/hyracks-storage-am-common/pom.xml
+++ b/hyracks-storage-am-common/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-storage-am-common</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -27,28 +27,28 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-api</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java
index 8b5a325..c33a8d8 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java
@@ -59,7 +59,7 @@
     public ByteBuffer getBuffer();
     
     // for debugging
-    public void printHeader();
+    public String printHeader();
 
     public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey) throws TreeIndexException;
 
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexMetaDataFrame.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexMetaDataFrame.java
index 4d81e5e6..17519ae 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexMetaDataFrame.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexMetaDataFrame.java
@@ -18,7 +18,7 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 
 public interface ITreeIndexMetaDataFrame {
-	public void initBuffer(int level);
+	public void initBuffer(byte level);
 
 	public void setPage(ICachedPage page);
 
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/LIFOMetaDataFrame.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/LIFOMetaDataFrame.java
index 0e4a6e7..c87b84a 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/LIFOMetaDataFrame.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/LIFOMetaDataFrame.java
@@ -27,12 +27,11 @@
 
 public class LIFOMetaDataFrame implements ITreeIndexMetaDataFrame {
 
-	protected static final int tupleCountOff = 0; // 0
-	protected static final int freeSpaceOff = tupleCountOff + 8; // 8 
-	protected static final int maxPageOff = freeSpaceOff + 4; // 12
-	protected static final int dummyFieldOff = maxPageOff + 4; // 16
-	protected static final byte levelOff = dummyFieldOff + 4; // 20
-	protected static final byte nextPageOff = levelOff + 1;
+	protected static final int tupleCountOff = 0; //0
+	protected static final int freeSpaceOff = tupleCountOff + 4; //4
+	protected static final int maxPageOff = freeSpaceOff + 4; //8
+	protected static final int levelOff = maxPageOff + 12; //20
+	protected static final int nextPageOff = levelOff + 1; // 21
 
 	protected ICachedPage page = null;
 	protected ByteBuffer buf = null;
@@ -96,10 +95,11 @@
 	}
 
 	@Override
-	public void initBuffer(int level) {
-		buf.putInt(freeSpaceOff, nextPageOff + 4);
+	public void initBuffer(byte level) {
 		buf.putInt(tupleCountOff, 0);
-		buf.putInt(levelOff, level);
+		buf.putInt(freeSpaceOff, nextPageOff + 4);
+		//buf.putInt(maxPageOff, -1);
+		buf.put(levelOff, level);
 		buf.putInt(nextPageOff, -1);
 	}
 
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
index 197af60..e2e28fd 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
@@ -32,9 +32,9 @@
     protected static final int pageLsnOff = 0; // 0
     protected static final int tupleCountOff = pageLsnOff + 8; // 8
     protected static final int freeSpaceOff = tupleCountOff + 4; // 12
-    protected static final int totalFreeSpaceOff = freeSpaceOff + 4; // 20
-    protected static final byte levelOff = totalFreeSpaceOff + 4; // 24
-    protected static final byte smFlagOff = levelOff + 1; // 25
+    protected static final int totalFreeSpaceOff = freeSpaceOff + 4; // 16
+    protected static final int levelOff = totalFreeSpaceOff + 4; // 20
+    protected static final int smFlagOff = levelOff + 1; // 21
 
     protected ICachedPage page = null;
     protected ByteBuffer buf = null;
@@ -228,9 +228,15 @@
     }
 
     @Override
-    public void printHeader() {
-        // TODO Auto-generated method stub
-
+    public String printHeader() {
+    	StringBuilder strBuilder = new StringBuilder();
+    	strBuilder.append("pageLsnOff:        " + pageLsnOff + "\n");
+    	strBuilder.append("tupleCountOff:     " + tupleCountOff + "\n");
+    	strBuilder.append("freeSpaceOff:      " + freeSpaceOff + "\n");
+    	strBuilder.append("totalFreeSpaceOff: " + totalFreeSpaceOff + "\n");
+    	strBuilder.append("levelOff:          " + levelOff + "\n");
+    	strBuilder.append("smFlagOff:         " + smFlagOff + "\n");
+    	return strBuilder.toString();
     }
 
     @Override
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/TreeIndexUtils.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/TreeIndexUtils.java
index 5c16665..a1e493d 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/TreeIndexUtils.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/TreeIndexUtils.java
@@ -30,7 +30,7 @@
 			tuple.resetByTupleIndex(frame, i);
 			String tupleString = TupleUtils.printTuple(tuple, fieldSerdes);
 			strBuilder.append(tupleString);
-			if (i != tuple.getFieldCount() - 1) {
+			if (i != frame.getTupleCount() - 1) {
 				strBuilder.append(" | ");
 			}
 		}
diff --git a/hyracks-storage-am-invertedindex/pom.xml b/hyracks-storage-am-invertedindex/pom.xml
index 4578a46..32ce735 100644
--- a/hyracks-storage-am-invertedindex/pom.xml
+++ b/hyracks-storage-am-invertedindex/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-storage-am-invertedindex</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -27,35 +27,35 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  	
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  	
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  	
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-nc</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  	
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-am-btree</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>    
diff --git a/hyracks-storage-am-rtree/pom.xml b/hyracks-storage-am-rtree/pom.xml
index 10f118cf..9abdd00 100644
--- a/hyracks-storage-am-rtree/pom.xml
+++ b/hyracks-storage-am-rtree/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-storage-am-rtree</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -27,28 +27,28 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-am-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  	
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  	
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  	
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-nc</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>  		
diff --git a/hyracks-storage-common/pom.xml b/hyracks-storage-common/pom.xml
index ff6887d..50eecd2 100644
--- a/hyracks-storage-common/pom.xml
+++ b/hyracks-storage-common/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-storage-common</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -27,7 +27,7 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-api</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
diff --git a/hyracks-test-support/pom.xml b/hyracks-test-support/pom.xml
index e9a8a27..95b3937 100644
--- a/hyracks-test-support/pom.xml
+++ b/hyracks-test-support/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-test-support</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -27,19 +27,19 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-nc</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-am-btree</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
index 295a428..1b95ccb 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
@@ -42,7 +42,7 @@
     }
 
     @Override
-    public Serializable getDestributedState() {
+    public Serializable getDistributedState() {
         return distributedState;
     }
 
@@ -52,11 +52,6 @@
     }
 
     @Override
-    public void setDistributedState(Serializable state) {
-        distributedState = state;
-    }
-
-    @Override
     public void setApplicationObject(Object object) {
         this.appObject = object;
     }
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
index 83b608f..cbab080 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
@@ -18,7 +18,6 @@
 
 import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
@@ -89,21 +88,4 @@
     public TaskAttemptId getTaskAttemptId() {
         return taskId;
     }
-
-    @Override
-    public Object lookupGlobalVariable(ActivityId producerActivity, int partition, String varName)
-            throws HyracksDataException {
-        return null;
-    }
-
-    @Override
-    public Object lookupLocalVariable(ActivityId producerActivity, int partition, String varName)
-            throws HyracksDataException {
-        return null;
-    }
-
-    @Override
-    public void setVariable(String name, Object value) throws HyracksDataException {
-
-    }
 }
\ No newline at end of file
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/pom.xml b/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
index 94089f0..68ab309 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
+++ b/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-storage-am-btree-test</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks-tests</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -34,20 +34,20 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-nc</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-am-btree</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-test-support</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>test</scope>
   	</dependency>
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
index 3b59e6f..7ca504b 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
@@ -726,6 +726,7 @@
 
             LOGGER.info("INSERTING INTO BTREE");
             int maxLength = 10;
+            //int ins = 16;
             int ins = 10000;
             String[] f0s = new String[ins];
             String[] f1s = new String[ins];
@@ -757,16 +758,14 @@
                     btree.insert(tuple, insertOpCtx);
                     insDone++;
                 } catch (TreeIndexException e) {
-                    // e.printStackTrace();
+                    //e.printStackTrace();
                 } catch (Exception e) {
                     e.printStackTrace();
                 }
 
                 insDoneCmp[i] = insDone;
             }
-            // btree.printTree();
-            // btree.printStats();
-
+            
             LOGGER.info("DELETING FROM BTREE");
             int delDone = 0;
             for (int i = 0; i < ins; i++) {
@@ -790,7 +789,7 @@
                     btree.delete(tuple, deleteOpCtx);
                     delDone++;
                 } catch (TreeIndexException e) {
-                    // e.printStackTrace();
+                    //e.printStackTrace();
                 } catch (Exception e) {
                     e.printStackTrace();
                 }
@@ -1181,7 +1180,7 @@
     // TIME-INTERVAL INTERSECTION DEMO FOR EVENT PEOPLE
     // demo for Arjun to show easy support of intersection queries on
     // time-intervals
-    //@Test
+    @Test
     public void test07() throws Exception {
 
         LOGGER.info("TIME-INTERVAL INTERSECTION DEMO");
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml b/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
index c098506..7d40c81 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-storage-am-invertedindex-test</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks-tests</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -27,20 +27,20 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-nc</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-am-invertedindex</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-test-support</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>test</scope>
   	</dependency>
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml b/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
index cc02dca..bd1b82e 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-storage-am-rtree-test</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks-tests</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -34,20 +34,20 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-nc</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-am-rtree</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-test-support</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>test</scope>
   	</dependency>
diff --git a/hyracks-tests/hyracks-storage-common-test/pom.xml b/hyracks-tests/hyracks-storage-common-test/pom.xml
index ca68f85..cbe34ef 100644
--- a/hyracks-tests/hyracks-storage-common-test/pom.xml
+++ b/hyracks-tests/hyracks-storage-common-test/pom.xml
@@ -2,12 +2,12 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-storage-common-test</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks-tests</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <build>
@@ -34,19 +34,19 @@
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-control-nc</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-storage-common</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-test-support</artifactId>
-  		<version>0.1.8-SNAPSHOT</version>
+  		<version>0.2.0-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>test</scope>
   	</dependency>
diff --git a/hyracks-tests/pom.xml b/hyracks-tests/pom.xml
index 13abbd7..b120ba82 100644
--- a/hyracks-tests/pom.xml
+++ b/hyracks-tests/pom.xml
@@ -2,13 +2,13 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-tests</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
-    <version>0.1.8-SNAPSHOT</version>
+    <version>0.2.0-SNAPSHOT</version>
   </parent>
 
   <modules>
diff --git a/pom.xml b/pom.xml
index 6771a7a..2200b23 100644
--- a/pom.xml
+++ b/pom.xml
@@ -2,7 +2,7 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks</artifactId>
-  <version>0.1.8-SNAPSHOT</version>
+  <version>0.2.0-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <build>
@@ -15,6 +15,11 @@
             <goals>package source:jar javadoc:jar deploy:deploy</goals>
         </configuration>
       </plugin>
+      <plugin>
+      	<groupId>org.codehaus.mojo</groupId>
+      	<artifactId>versions-maven-plugin</artifactId>
+      	<version>1.2</version>
+      </plugin>
     </plugins>
   </build>