merge hyracks_dev_next r978:1064

git-svn-id: https://hyracks.googlecode.com/svn/branches/aggregators_dev_next@1065 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
index 06b76ef..922b0fd 100644
--- a/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
@@ -81,9 +81,9 @@
     public static void runJob(JobSpecification spec) throws Exception {
         JobId jobId = hcc.createJob(AlgebricksConfig.HYRACKS_APP_NAME, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
         AlgebricksConfig.ALGEBRICKS_LOGGER.info(spec.toJSON().toString());
-        cc.startJob(jobId);
+        hcc.start(jobId);
         AlgebricksConfig.ALGEBRICKS_LOGGER.info(jobId.toString());
-        cc.waitForCompletion(jobId);
+        hcc.waitForCompletion(jobId);
     }
 
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
index f74d06e..602288c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
@@ -21,82 +21,76 @@
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.SyncRMI;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
 
 public class HyracksClientInterfaceRemoteProxy implements IHyracksClientInterface {
     private final IIPCHandle ipcHandle;
 
-    public HyracksClientInterfaceRemoteProxy(IIPCHandle ipcHandle) {
+    private final RPCInterface rpci;
+
+    public HyracksClientInterfaceRemoteProxy(IIPCHandle ipcHandle, RPCInterface rpci) {
         this.ipcHandle = ipcHandle;
+        this.rpci = rpci;
     }
 
     @Override
     public ClusterControllerInfo getClusterControllerInfo() throws Exception {
-        SyncRMI sync = new SyncRMI();
         HyracksClientInterfaceFunctions.GetClusterControllerInfoFunction gccif = new HyracksClientInterfaceFunctions.GetClusterControllerInfoFunction();
-        return (ClusterControllerInfo) sync.call(ipcHandle, gccif);
+        return (ClusterControllerInfo) rpci.call(ipcHandle, gccif);
     }
 
     @Override
     public void createApplication(String appName) throws Exception {
-        SyncRMI sync = new SyncRMI();
         HyracksClientInterfaceFunctions.CreateApplicationFunction caf = new HyracksClientInterfaceFunctions.CreateApplicationFunction(
                 appName);
-        sync.call(ipcHandle, caf);
+        rpci.call(ipcHandle, caf);
     }
 
     @Override
     public void startApplication(String appName) throws Exception {
-        SyncRMI sync = new SyncRMI();
         HyracksClientInterfaceFunctions.StartApplicationFunction saf = new HyracksClientInterfaceFunctions.StartApplicationFunction(
                 appName);
-        sync.call(ipcHandle, saf);
+        rpci.call(ipcHandle, saf);
     }
 
     @Override
     public void destroyApplication(String appName) throws Exception {
-        SyncRMI sync = new SyncRMI();
         HyracksClientInterfaceFunctions.DestroyApplicationFunction daf = new HyracksClientInterfaceFunctions.DestroyApplicationFunction(
                 appName);
-        sync.call(ipcHandle, daf);
+        rpci.call(ipcHandle, daf);
     }
 
     @Override
     public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
-        SyncRMI sync = new SyncRMI();
         HyracksClientInterfaceFunctions.CreateJobFunction cjf = new HyracksClientInterfaceFunctions.CreateJobFunction(
                 appName, jobSpec, jobFlags);
-        return (JobId) sync.call(ipcHandle, cjf);
+        return (JobId) rpci.call(ipcHandle, cjf);
     }
 
     @Override
     public JobStatus getJobStatus(JobId jobId) throws Exception {
-        SyncRMI sync = new SyncRMI();
         HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf = new HyracksClientInterfaceFunctions.GetJobStatusFunction(
                 jobId);
-        return (JobStatus) sync.call(ipcHandle, gjsf);
+        return (JobStatus) rpci.call(ipcHandle, gjsf);
     }
 
     @Override
     public void startJob(JobId jobId) throws Exception {
-        SyncRMI sync = new SyncRMI();
         HyracksClientInterfaceFunctions.StartJobFunction sjf = new HyracksClientInterfaceFunctions.StartJobFunction(
                 jobId);
-        sync.call(ipcHandle, sjf);
+        rpci.call(ipcHandle, sjf);
     }
 
     @Override
     public void waitForCompletion(JobId jobId) throws Exception {
-        SyncRMI sync = new SyncRMI();
         HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = new HyracksClientInterfaceFunctions.WaitForCompletionFunction(
                 jobId);
-        sync.call(ipcHandle, wfcf);
+        rpci.call(ipcHandle, wfcf);
     }
 
     @Override
     public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception {
-        SyncRMI sync = new SyncRMI();
         HyracksClientInterfaceFunctions.GetNodeControllersInfoFunction gncif = new HyracksClientInterfaceFunctions.GetNodeControllersInfoFunction();
-        return (Map<String, NodeControllerInfo>) sync.call(ipcHandle, gncif);
+        return (Map<String, NodeControllerInfo>) rpci.call(ipcHandle, gncif);
     }
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
index 1b38cb9..8c9409c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
@@ -32,7 +32,9 @@
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
 import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
 import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
+import edu.uci.ics.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
 
 /**
  * Connection Class used by a Hyracks Client to interact with a Hyracks Cluster
@@ -63,10 +65,11 @@
      */
     public HyracksConnection(String ccHost, int ccPort) throws Exception {
         this.ccHost = ccHost;
-        ipc = new IPCSystem(new InetSocketAddress(0));
+        RPCInterface rpci = new RPCInterface();
+        ipc = new IPCSystem(new InetSocketAddress(0), rpci, new JavaSerializationBasedPayloadSerializerDeserializer());
         ipc.start();
         IIPCHandle ccIpchandle = ipc.getHandle(new InetSocketAddress(ccHost, ccPort));
-        this.hci = new HyracksClientInterfaceRemoteProxy(ccIpchandle);
+        this.hci = new HyracksClientInterfaceRemoteProxy(ccIpchandle, rpci);
         ccInfo = hci.getClusterControllerInfo();
     }
 
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
index 1424a63..703f74b 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
@@ -15,20 +15,16 @@
 package edu.uci.ics.hyracks.api.client;
 
 import java.io.Serializable;
-import java.net.InetAddress;
 
 public class NodeControllerInfo implements Serializable {
     private static final long serialVersionUID = 1L;
 
     private final String nodeId;
 
-    private final InetAddress ipAddress;
-
     private final NodeStatus status;
 
-    public NodeControllerInfo(String nodeId, InetAddress ipAddress, NodeStatus status) {
+    public NodeControllerInfo(String nodeId, NodeStatus status) {
         this.nodeId = nodeId;
-        this.ipAddress = ipAddress;
         this.status = status;
     }
 
@@ -36,10 +32,6 @@
         return nodeId;
     }
 
-    public InetAddress getIpAddress() {
-        return ipAddress;
-    }
-
     public NodeStatus getStatus() {
         return status;
     }
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
index 868221d..d176c3c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
@@ -15,21 +15,20 @@
 package edu.uci.ics.hyracks.api.comm;
 
 import java.io.Serializable;
-import java.net.InetAddress;
 
 public final class NetworkAddress implements Serializable {
     private static final long serialVersionUID = 1L;
 
-    private final InetAddress ipAddress;
+    private final byte[] ipAddress;
 
     private final int port;
 
-    public NetworkAddress(InetAddress ipAddress, int port) {
+    public NetworkAddress(byte[] ipAddress, int port) {
         this.ipAddress = ipAddress;
         this.port = port;
     }
 
-    public InetAddress getIpAddress() {
+    public byte[] getIpAddress() {
         return ipAddress;
     }
 
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/ICCContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/ICCContext.java
index 266ebe9..abd9c20 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/ICCContext.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/ICCContext.java
@@ -18,5 +18,5 @@
 import java.util.Set;
 
 public interface ICCContext {
-    public Map<String, Set<String>> getIPAddressNodeMap();
+    public void getIPAddressNodeMap(Map<String, Set<String>> map) throws Exception;
 }
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
index 75cc245..41b4c23 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
@@ -19,9 +19,9 @@
 public final class ActivityId implements Serializable {
     private static final long serialVersionUID = 1L;
     private final OperatorDescriptorId odId;
-    private final long id;
+    private final int id;
 
-    public ActivityId(OperatorDescriptorId odId, long id) {
+    public ActivityId(OperatorDescriptorId odId, int id) {
         this.odId = odId;
         this.id = id;
     }
@@ -30,7 +30,7 @@
         return odId;
     }
 
-    public long getLocalId() {
+    public int getLocalId() {
         return id;
     }
 
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
index ec8ab76..c816d26 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
@@ -44,7 +44,8 @@
     public ConnectorDescriptorId getConnectorId();
 
     /**
-     * Factory method to create the send side writer that writes into this connector.
+     * Factory method to create the send side writer that writes into this
+     * connector.
      * 
      * @param ctx
      *            Context
@@ -66,7 +67,8 @@
             throws HyracksDataException;
 
     /**
-     * Factory metod to create the receive side reader that reads data from this connector.
+     * Factory metod to create the receive side reader that reads data from this
+     * connector.
      * 
      * @param ctx
      *            Context
@@ -96,12 +98,23 @@
             ICCApplicationContext appCtx);
 
     /**
-     * Indicate which consumer partitions may receive data from the given producer partition.
+     * Indicate which consumer partitions may receive data from the given
+     * producer partition.
      */
     public void indicateTargetPartitions(int nProducerPartitions, int nConsumerPartitions, int producerIndex,
             BitSet targetBitmap);
 
     /**
+     * Gets the display name.
+     */
+    public String getDisplayName();
+
+    /**
+     * Sets the display name.
+     */
+    public void setDisplayName(String displayName);
+
+    /**
      * Translate this connector descriptor to JSON.
      * 
      * @return
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
index 78847eb..c37a530 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
@@ -59,7 +59,8 @@
     public RecordDescriptor[] getOutputRecordDescriptors();
 
     /**
-     * Contributes the activity graph that describes the behavior of this operator.
+     * Contributes the activity graph that describes the behavior of this
+     * operator.
      * 
      * @param builder
      *            - graph builder
@@ -78,6 +79,16 @@
             ICCApplicationContext appCtx);
 
     /**
+     * Gets the display name.
+     */
+    public String getDisplayName();
+
+    /**
+     * Sets the display name.
+     */
+    public void setDisplayName(String displayName);
+
+    /**
      * Translates this operator descriptor to JSON.
      */
     public JSONObject toJSON() throws JSONException;
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 4575adc..d3e472b 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
@@ -16,11 +16,9 @@
 
 import java.io.File;
 import java.net.InetSocketAddress;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Hashtable;
 import java.util.LinkedHashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.Timer;
@@ -31,22 +29,19 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
-import edu.uci.ics.hyracks.api.client.IHyracksClientInterface;
+import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
 import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
 import edu.uci.ics.hyracks.api.context.ICCContext;
-import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
-import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobStatus;
 import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.cc.ipc.HyracksClientInterfaceDelegateIPCI;
-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.web.WebServer;
 import edu.uci.ics.hyracks.control.cc.work.ApplicationCreateWork;
 import edu.uci.ics.hyracks.control.cc.work.ApplicationDestroyWork;
 import edu.uci.ics.hyracks.control.cc.work.ApplicationStartWork;
-import edu.uci.ics.hyracks.control.cc.work.GetJobStatusConditionVariableWork;
+import edu.uci.ics.hyracks.control.cc.work.ApplicationStateChangeWork;
+import edu.uci.ics.hyracks.control.cc.work.GetIpAddressNodeNameMapWork;
 import edu.uci.ics.hyracks.control.cc.work.GetJobStatusWork;
 import edu.uci.ics.hyracks.control.cc.work.GetNodeControllersInfoWork;
 import edu.uci.ics.hyracks.control.cc.work.JobCreateWork;
@@ -61,29 +56,22 @@
 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.cc.work.WaitForJobCompletionWork;
 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;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
 import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
-import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
-import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
-import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
-import edu.uci.ics.hyracks.control.common.ipc.ClusterControllerDelegateIPCI;
-import edu.uci.ics.hyracks.control.common.ipc.NodeControllerRemoteProxy;
-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.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
+import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions;
+import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions.Function;
 import edu.uci.ics.hyracks.control.common.logs.LogFile;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.IPCResponder;
 import edu.uci.ics.hyracks.control.common.work.WorkQueue;
 import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
 import edu.uci.ics.hyracks.ipc.api.IIPCI;
+import edu.uci.ics.hyracks.ipc.exceptions.IPCException;
 import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
+import edu.uci.ics.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
 
-public class ClusterControllerService extends AbstractRemoteService implements IClusterController,
-        IHyracksClientInterface {
+public class ClusterControllerService extends AbstractRemoteService {
     private static Logger LOGGER = Logger.getLogger(ClusterControllerService.class.getName());
 
     private final CCConfig ccConfig;
@@ -131,11 +119,12 @@
         applications = new Hashtable<String, CCApplicationContext>();
         serverCtx = new ServerContext(ServerContext.ServerType.CLUSTER_CONTROLLER, new File(ccConfig.ccRoot));
         executor = Executors.newCachedThreadPool();
-        IIPCI ccIPCI = new ClusterControllerDelegateIPCI(this);
-        clusterIPC = new IPCSystem(new InetSocketAddress(ccConfig.clusterNetPort), ccIPCI, executor);
-        IIPCI ciIPCI = new HyracksClientInterfaceDelegateIPCI(this);
+        IIPCI ccIPCI = new ClusterControllerIPCI();
+        clusterIPC = new IPCSystem(new InetSocketAddress(ccConfig.clusterNetPort), ccIPCI,
+                new CCNCFunctions.SerializerDeserializer());
+        IIPCI ciIPCI = new HyracksClientInterfaceIPCI();
         clientIPC = new IPCSystem(new InetSocketAddress(ccConfig.clientNetIpAddress, ccConfig.clientNetPort), ciIPCI,
-                executor);
+                new JavaSerializationBasedPayloadSerializerDeserializer());
         webServer = new WebServer(this);
         activeRunMap = new HashMap<JobId, JobRun>();
         runMapArchive = new LinkedHashMap<JobId, JobRun>() {
@@ -149,8 +138,9 @@
         this.timer = new Timer(true);
         ccContext = new ICCContext() {
             @Override
-            public Map<String, Set<String>> getIPAddressNodeMap() {
-                return ipAddressNodeNameMap;
+            public void getIPAddressNodeMap(Map<String, Set<String>> map) throws Exception {
+                GetIpAddressNodeNameMapWork ginmw = new GetIpAddressNodeNameMapWork(ClusterControllerService.this, map);
+                workQueue.scheduleAndSync(ginmw);
             }
         };
         sweeper = new DeadNodeSweeper();
@@ -202,6 +192,10 @@
         return runMapArchive;
     }
 
+    public Map<String, Set<String>> getIpAddressNodeNameMap() {
+        return ipAddressNodeNameMap;
+    }
+
     public LogFile getJobLogFile() {
         return jobLog;
     }
@@ -226,131 +220,16 @@
         return new JobId(jobCounter++);
     }
 
-    @Override
-    public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
-        JobId jobId = createJobId();
-        JobCreateWork jce = new JobCreateWork(this, jobId, appName, jobSpec, jobFlags);
-        workQueue.schedule(jce);
-        jce.sync();
-        return jobId;
-    }
-
-    @Override
-    public NodeParameters registerNode(NodeRegistration reg) throws Exception {
-        InetSocketAddress ncAddress = reg.getNodeControllerAddress();
-        String id = reg.getNodeId();
-
-        IIPCHandle ncIPCHandle = clusterIPC.getHandle(reg.getNodeControllerAddress());
-        INodeController nodeController = new NodeControllerRemoteProxy(ncIPCHandle);
-
-        NodeControllerState state = new NodeControllerState(nodeController, reg);
-        workQueue.scheduleAndSync(new RegisterNodeWork(this, id, state));
-        LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
-        NodeParameters params = new NodeParameters();
-        params.setClusterControllerInfo(info);
-        params.setHeartbeatPeriod(ccConfig.heartbeatPeriod);
-        params.setProfileDumpPeriod(ccConfig.profileDumpPeriod);
-        return params;
-    }
-
-    @Override
-    public void unregisterNode(String nodeId) throws Exception {
-        workQueue.schedule(new UnregisterNodeWork(this, nodeId));
-    }
-
-    @Override
-    public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
-            throws Exception {
-        TaskCompleteWork sce = new TaskCompleteWork(this, jobId, taskId, nodeId, statistics);
-        workQueue.schedule(sce);
-    }
-
-    @Override
-    public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, String details) throws Exception {
-        TaskFailureWork tfe = new TaskFailureWork(this, jobId, taskId, nodeId, details);
-        workQueue.schedule(tfe);
-    }
-
-    @Override
-    public void notifyJobletCleanup(JobId jobId, String nodeId) throws Exception {
-        JobletCleanupNotificationWork jcnw = new JobletCleanupNotificationWork(this, jobId, nodeId);
-        workQueue.schedule(jcnw);
-    }
-
-    @Override
-    public JobStatus getJobStatus(JobId jobId) throws Exception {
-        GetJobStatusWork gse = new GetJobStatusWork(this, jobId);
-        workQueue.scheduleAndSync(gse);
-        return gse.getStatus();
-    }
-
-    @Override
-    public void startJob(JobId jobId) throws Exception {
-        JobStartWork jse = new JobStartWork(this, jobId);
-        workQueue.schedule(jse);
-    }
-
-    @Override
-    public void waitForCompletion(JobId jobId) throws Exception {
-        GetJobStatusConditionVariableWork e = new GetJobStatusConditionVariableWork(this, jobId);
-        workQueue.scheduleAndSync(e);
-        IJobStatusConditionVariable var = e.getConditionVariable();
-        if (var != null) {
-            var.waitForCompletion();
-        }
-    }
-
-    @Override
-    public void reportProfile(String id, List<JobProfile> profiles) throws Exception {
-        workQueue.schedule(new ReportProfilesWork(this, profiles));
-    }
-
-    @Override
-    public synchronized void nodeHeartbeat(String id, HeartbeatData hbData) throws Exception {
-        workQueue.schedule(new NodeHeartbeatWork(this, id, hbData));
-    }
-
-    @Override
-    public void createApplication(String appName) throws Exception {
-        FutureValue<Object> fv = new FutureValue<Object>();
-        workQueue.schedule(new ApplicationCreateWork(this, appName, fv));
-        fv.get();
-    }
-
-    @Override
-    public void destroyApplication(String appName) throws Exception {
-        FutureValue<Object> fv = new FutureValue<Object>();
-        workQueue.schedule(new ApplicationDestroyWork(this, appName, fv));
-        fv.get();
-    }
-
-    @Override
-    public void startApplication(final String appName) throws Exception {
-        FutureValue<Object> fv = new FutureValue<Object>();
-        workQueue.schedule(new ApplicationStartWork(this, appName, fv));
-        fv.get();
-    }
-
-    @Override
-    public ClusterControllerInfo getClusterControllerInfo() throws Exception {
+    public ClusterControllerInfo getClusterControllerInfo() {
         return info;
     }
 
-    @Override
-    public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception {
-        FutureValue<Map<String, NodeControllerInfo>> fv = new FutureValue<Map<String, NodeControllerInfo>>();
-        workQueue.schedule(new GetNodeControllersInfoWork(this, fv));
-        return fv.get();
+    public CCConfig getCCConfig() {
+        return ccConfig;
     }
 
-    @Override
-    public void registerPartitionProvider(PartitionDescriptor partitionDescriptor) {
-        workQueue.schedule(new RegisterPartitionAvailibilityWork(this, partitionDescriptor));
-    }
-
-    @Override
-    public void registerPartitionRequest(PartitionRequest partitionRequest) {
-        workQueue.schedule(new RegisterPartitionRequestWork(this, partitionRequest));
+    public IPCSystem getClusterIPC() {
+        return clusterIPC;
     }
 
     private class DeadNodeSweeper extends TimerTask {
@@ -359,4 +238,156 @@
             workQueue.schedule(new RemoveDeadNodesWork(ClusterControllerService.this));
         }
     }
+
+    private class HyracksClientInterfaceIPCI implements IIPCI {
+        @Override
+        public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception) {
+            HyracksClientInterfaceFunctions.Function fn = (HyracksClientInterfaceFunctions.Function) payload;
+            switch (fn.getFunctionId()) {
+                case GET_CLUSTER_CONTROLLER_INFO: {
+                    try {
+                        handle.send(mid, info, null);
+                    } catch (IPCException e) {
+                        e.printStackTrace();
+                    }
+                    return;
+                }
+
+                case CREATE_APPLICATION: {
+                    HyracksClientInterfaceFunctions.CreateApplicationFunction caf = (HyracksClientInterfaceFunctions.CreateApplicationFunction) fn;
+                    workQueue.schedule(new ApplicationCreateWork(ClusterControllerService.this, caf.getAppName(),
+                            new IPCResponder<Object>(handle, mid)));
+                    return;
+                }
+
+                case START_APPLICATION: {
+                    HyracksClientInterfaceFunctions.StartApplicationFunction saf = (HyracksClientInterfaceFunctions.StartApplicationFunction) fn;
+                    workQueue.schedule(new ApplicationStartWork(ClusterControllerService.this, saf.getAppName(),
+                            new IPCResponder<Object>(handle, mid)));
+                    return;
+                }
+
+                case DESTROY_APPLICATION: {
+                    HyracksClientInterfaceFunctions.DestroyApplicationFunction daf = (HyracksClientInterfaceFunctions.DestroyApplicationFunction) fn;
+                    workQueue.schedule(new ApplicationDestroyWork(ClusterControllerService.this, daf.getAppName(),
+                            new IPCResponder<Object>(handle, mid)));
+                    return;
+                }
+
+                case CREATE_JOB: {
+                    HyracksClientInterfaceFunctions.CreateJobFunction cjf = (HyracksClientInterfaceFunctions.CreateJobFunction) fn;
+                    JobId jobId = createJobId();
+                    workQueue.schedule(new JobCreateWork(ClusterControllerService.this, jobId, cjf.getAppName(), cjf
+                            .getJobSpec(), cjf.getJobFlags(), new IPCResponder<JobId>(handle, mid)));
+                    return;
+                }
+
+                case GET_JOB_STATUS: {
+                    HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf = (HyracksClientInterfaceFunctions.GetJobStatusFunction) fn;
+                    workQueue.schedule(new GetJobStatusWork(ClusterControllerService.this, gjsf.getJobId(),
+                            new IPCResponder<JobStatus>(handle, mid)));
+                    return;
+                }
+
+                case START_JOB: {
+                    HyracksClientInterfaceFunctions.StartJobFunction sjf = (HyracksClientInterfaceFunctions.StartJobFunction) fn;
+                    workQueue.schedule(new JobStartWork(ClusterControllerService.this, sjf.getJobId(),
+                            new IPCResponder<Object>(handle, mid)));
+                    return;
+                }
+
+                case WAIT_FOR_COMPLETION: {
+                    HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = (HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
+                    workQueue.schedule(new WaitForJobCompletionWork(ClusterControllerService.this, wfcf.getJobId(),
+                            new IPCResponder<Object>(handle, mid)));
+                    return;
+                }
+
+                case GET_NODE_CONTROLLERS_INFO: {
+                    workQueue.schedule(new GetNodeControllersInfoWork(ClusterControllerService.this,
+                            new IPCResponder<Map<String, NodeControllerInfo>>(handle, mid)));
+                    return;
+                }
+            }
+            try {
+                handle.send(mid, null, new IllegalArgumentException("Unknown function " + fn.getFunctionId()));
+            } catch (IPCException e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    private class ClusterControllerIPCI implements IIPCI {
+        @Override
+        public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception) {
+            CCNCFunctions.Function fn = (Function) payload;
+            switch (fn.getFunctionId()) {
+                case REGISTER_NODE: {
+                    CCNCFunctions.RegisterNodeFunction rnf = (CCNCFunctions.RegisterNodeFunction) fn;
+                    workQueue.schedule(new RegisterNodeWork(ClusterControllerService.this, rnf.getNodeRegistration()));
+                    return;
+                }
+
+                case UNREGISTER_NODE: {
+                    CCNCFunctions.UnregisterNodeFunction unf = (CCNCFunctions.UnregisterNodeFunction) fn;
+                    workQueue.schedule(new UnregisterNodeWork(ClusterControllerService.this, unf.getNodeId()));
+                    return;
+                }
+
+                case NODE_HEARTBEAT: {
+                    CCNCFunctions.NodeHeartbeatFunction nhf = (CCNCFunctions.NodeHeartbeatFunction) fn;
+                    workQueue.schedule(new NodeHeartbeatWork(ClusterControllerService.this, nhf.getNodeId(), nhf
+                            .getHeartbeatData()));
+                    return;
+                }
+
+                case NOTIFY_JOBLET_CLEANUP: {
+                    CCNCFunctions.NotifyJobletCleanupFunction njcf = (CCNCFunctions.NotifyJobletCleanupFunction) fn;
+                    workQueue.schedule(new JobletCleanupNotificationWork(ClusterControllerService.this,
+                            njcf.getJobId(), njcf.getNodeId()));
+                    return;
+                }
+
+                case REPORT_PROFILE: {
+                    CCNCFunctions.ReportProfileFunction rpf = (CCNCFunctions.ReportProfileFunction) fn;
+                    workQueue.schedule(new ReportProfilesWork(ClusterControllerService.this, rpf.getProfiles()));
+                    return;
+                }
+
+                case NOTIFY_TASK_COMPLETE: {
+                    CCNCFunctions.NotifyTaskCompleteFunction ntcf = (CCNCFunctions.NotifyTaskCompleteFunction) fn;
+                    workQueue.schedule(new TaskCompleteWork(ClusterControllerService.this, ntcf.getJobId(), ntcf
+                            .getTaskId(), ntcf.getNodeId(), ntcf.getStatistics()));
+                    return;
+                }
+                case NOTIFY_TASK_FAILURE: {
+                    CCNCFunctions.NotifyTaskFailureFunction ntff = (CCNCFunctions.NotifyTaskFailureFunction) fn;
+                    workQueue.schedule(new TaskFailureWork(ClusterControllerService.this, ntff.getJobId(), ntff
+                            .getTaskId(), ntff.getDetails(), ntff.getDetails()));
+                    return;
+                }
+
+                case REGISTER_PARTITION_PROVIDER: {
+                    CCNCFunctions.RegisterPartitionProviderFunction rppf = (CCNCFunctions.RegisterPartitionProviderFunction) fn;
+                    workQueue.schedule(new RegisterPartitionAvailibilityWork(ClusterControllerService.this, rppf
+                            .getPartitionDescriptor()));
+                    return;
+                }
+
+                case REGISTER_PARTITION_REQUEST: {
+                    CCNCFunctions.RegisterPartitionRequestFunction rprf = (CCNCFunctions.RegisterPartitionRequestFunction) fn;
+                    workQueue.schedule(new RegisterPartitionRequestWork(ClusterControllerService.this, rprf
+                            .getPartitionRequest()));
+                    return;
+                }
+
+                case APPLICATION_STATE_CHANGE_RESPONSE: {
+                    CCNCFunctions.ApplicationStateChangeResponseFunction astrf = (CCNCFunctions.ApplicationStateChangeResponseFunction) fn;
+                    workQueue.schedule(new ApplicationStateChangeWork(ClusterControllerService.this, astrf));
+                    return;
+                }
+            }
+            LOGGER.warning("Unknown function: " + fn.getFunctionId());
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
index 0b1fea0..2b7563a 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
@@ -80,6 +80,14 @@
 
     private final long[][] gcCollectionTimes;
 
+    private final long[] netPayloadBytesRead;
+
+    private final long[] netPayloadBytesWritten;
+
+    private final long[] netSignalingBytesRead;
+
+    private final long[] netSignalingBytesWritten;
+
     private int rrdPtr;
 
     private int lastHeartbeatDuration;
@@ -116,6 +124,10 @@
         }
         gcCollectionCounts = new long[gcN][RRD_SIZE];
         gcCollectionTimes = new long[gcN][RRD_SIZE];
+        netPayloadBytesRead = new long[RRD_SIZE];
+        netPayloadBytesWritten = new long[RRD_SIZE];
+        netSignalingBytesRead = new long[RRD_SIZE];
+        netSignalingBytesWritten = new long[RRD_SIZE];
         rrdPtr = 0;
     }
 
@@ -139,6 +151,10 @@
             gcCollectionCounts[i][rrdPtr] = hbData.gcCollectionCounts[i];
             gcCollectionTimes[i][rrdPtr] = hbData.gcCollectionTimes[i];
         }
+        netPayloadBytesRead[rrdPtr] = hbData.netPayloadBytesRead;
+        netPayloadBytesWritten[rrdPtr] = hbData.netPayloadBytesWritten;
+        netSignalingBytesRead[rrdPtr] = hbData.netSignalingBytesRead;
+        netSignalingBytesWritten[rrdPtr] = hbData.netSignalingBytesWritten;
         rrdPtr = (rrdPtr + 1) % RRD_SIZE;
     }
 
@@ -199,6 +215,10 @@
         o.put("gc-names", gcNames);
         o.put("gc-collection-counts", gcCollectionCounts);
         o.put("gc-collection-times", gcCollectionTimes);
+        o.put("net-payload-bytes-read", netPayloadBytesRead);
+        o.put("net-payload-bytes-written", netPayloadBytesWritten);
+        o.put("net-signaling-bytes-read", netSignalingBytesRead);
+        o.put("net-signaling-bytes-written", netSignalingBytesWritten);
 
         return o;
     }
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
index c1c0161..28f7c59 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
@@ -17,7 +17,9 @@
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
 import edu.uci.ics.hyracks.api.application.ICCBootstrap;
@@ -30,10 +32,17 @@
 import edu.uci.ics.hyracks.control.cc.job.DeserializingJobSpecificationFactory;
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
 import edu.uci.ics.hyracks.control.common.context.ServerContext;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
 
 public class CCApplicationContext extends ApplicationContext implements ICCApplicationContext {
     private final ICCContext ccContext;
 
+    protected final Set<String> initPendingNodeIds;
+    protected final Set<String> deinitPendingNodeIds;
+
+    protected IResultCallback<Object> initializationCallback;
+    protected IResultCallback<Object> deinitializationCallback;
+
     private IJobSpecificationFactory jobSpecFactory;
 
     private List<IJobLifecycleListener> jobLifecycleListeners;
@@ -41,6 +50,8 @@
     public CCApplicationContext(ServerContext serverCtx, ICCContext ccContext, String appName) throws IOException {
         super(serverCtx, appName);
         this.ccContext = ccContext;
+        initPendingNodeIds = new HashSet<String>();
+        deinitPendingNodeIds = new HashSet<String>();
         jobSpecFactory = DeserializingJobSpecificationFactory.INSTANCE;
         jobLifecycleListeners = new ArrayList<IJobLifecycleListener>();
     }
@@ -98,4 +109,28 @@
             l.notifyJobCreation(jobId, specification);
         }
     }
+
+    public Set<String> getInitializationPendingNodeIds() {
+        return initPendingNodeIds;
+    }
+
+    public Set<String> getDeinitializationPendingNodeIds() {
+        return deinitPendingNodeIds;
+    }
+
+    public IResultCallback<Object> getInitializationCallback() {
+        return initializationCallback;
+    }
+
+    public void setInitializationCallback(IResultCallback<Object> initializationCallback) {
+        this.initializationCallback = initializationCallback;
+    }
+
+    public IResultCallback<Object> getDeinitializationCallback() {
+        return deinitializationCallback;
+    }
+
+    public void setDeinitializationCallback(IResultCallback<Object> deinitializationCallback) {
+        this.deinitializationCallback = deinitializationCallback;
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ipc/HyracksClientInterfaceDelegateIPCI.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ipc/HyracksClientInterfaceDelegateIPCI.java
deleted file mode 100644
index b3aa406..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ipc/HyracksClientInterfaceDelegateIPCI.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.ipc;
-
-import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
-import edu.uci.ics.hyracks.api.client.IHyracksClientInterface;
-import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.IIPCI;
-
-public class HyracksClientInterfaceDelegateIPCI implements IIPCI {
-    private final IHyracksClientInterface hci;
-
-    public HyracksClientInterfaceDelegateIPCI(IHyracksClientInterface hci) {
-        this.hci = hci;
-    }
-
-    @Override
-    public Object call(IIPCHandle caller, Object req) throws Exception {
-        HyracksClientInterfaceFunctions.Function fn = (HyracksClientInterfaceFunctions.Function) req;
-        switch (fn.getFunctionId()) {
-            case GET_CLUSTER_CONTROLLER_INFO: {
-                return hci.getClusterControllerInfo();
-            }
-
-            case CREATE_APPLICATION: {
-                HyracksClientInterfaceFunctions.CreateApplicationFunction caf = (HyracksClientInterfaceFunctions.CreateApplicationFunction) fn;
-                hci.createApplication(caf.getAppName());
-                return null;
-            }
-
-            case START_APPLICATION: {
-                HyracksClientInterfaceFunctions.StartApplicationFunction saf = (HyracksClientInterfaceFunctions.StartApplicationFunction) fn;
-                hci.startApplication(saf.getAppName());
-                return null;
-            }
-
-            case DESTROY_APPLICATION: {
-                HyracksClientInterfaceFunctions.DestroyApplicationFunction daf = (HyracksClientInterfaceFunctions.DestroyApplicationFunction) fn;
-                hci.destroyApplication(daf.getAppName());
-                return null;
-            }
-
-            case CREATE_JOB: {
-                HyracksClientInterfaceFunctions.CreateJobFunction cjf = (HyracksClientInterfaceFunctions.CreateJobFunction) fn;
-                return hci.createJob(cjf.getAppName(), cjf.getJobSpec(), cjf.getJobFlags());
-            }
-
-            case GET_JOB_STATUS: {
-                HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf = (HyracksClientInterfaceFunctions.GetJobStatusFunction) fn;
-                return hci.getJobStatus(gjsf.getJobId());
-            }
-
-            case START_JOB: {
-                HyracksClientInterfaceFunctions.StartJobFunction sjf = (HyracksClientInterfaceFunctions.StartJobFunction) fn;
-                hci.startJob(sjf.getJobId());
-                return null;
-            }
-
-            case WAIT_FOR_COMPLETION: {
-                HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = (HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
-                hci.waitForCompletion(wfcf.getJobId());
-                return null;
-            }
-
-            case GET_NODE_CONTROLLERS_INFO: {
-                return hci.getNodeControllersInfo();
-            }
-        }
-        throw new IllegalArgumentException("Unknown function " + fn.getFunctionId());
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java
deleted file mode 100644
index a37c786..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.remote;
-
-import java.util.List;
-import java.util.Vector;
-import java.util.concurrent.Semaphore;
-
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-public class RemoteRunner {
-    public static <T, R> R runRemote(ClusterControllerService ccs, final RemoteOp<T>[] remoteOps,
-            final Accumulator<T, R> accumulator) throws Exception {
-        final Semaphore installComplete = new Semaphore(remoteOps.length);
-        final List<Exception> errors = new Vector<Exception>();
-        for (final RemoteOp<T> remoteOp : remoteOps) {
-            NodeControllerState nodeState = ccs.getNodeMap().get(remoteOp.getNodeId());
-            final INodeController node = nodeState.getNodeController();
-
-            installComplete.acquire();
-            Runnable remoteRunner = new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        T t = remoteOp.execute(node);
-                        if (accumulator != null) {
-                            synchronized (accumulator) {
-                                accumulator.accumulate(t);
-                            }
-                        }
-                    } catch (Exception e) {
-                        errors.add(e);
-                    } finally {
-                        installComplete.release();
-                    }
-                }
-            };
-
-            ccs.getExecutor().execute(remoteRunner);
-        }
-        installComplete.acquire(remoteOps.length);
-        if (!errors.isEmpty()) {
-            throw errors.get(0);
-        }
-        return accumulator == null ? null : accumulator.getResult();
-    }
-}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java
deleted file mode 100644
index 52e726c..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.remote.ops;
-
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-public class ApplicationDestroyer implements RemoteOp<Void> {
-    private String nodeId;
-    private String appName;
-
-    public ApplicationDestroyer(String nodeId, String appName) {
-        this.nodeId = nodeId;
-        this.appName = appName;
-    }
-
-    @Override
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    @Override
-    public Void execute(INodeController node) throws Exception {
-        node.destroyApplication(appName);
-        return null;
-    }
-
-    @Override
-    public String toString() {
-        return "Destroyed application: " + appName;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java
deleted file mode 100644
index d6a1d26..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.remote.ops;
-
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-public class ApplicationStarter implements RemoteOp<Void> {
-    private String nodeId;
-    private String appName;
-    private boolean deployHar;
-    private byte[] distributedState;
-
-    public ApplicationStarter(String nodeId, String appName, boolean deployHar, byte[] distributedState) {
-        this.nodeId = nodeId;
-        this.appName = appName;
-        this.deployHar = deployHar;
-        this.distributedState = distributedState;
-    }
-
-    @Override
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    @Override
-    public Void execute(INodeController node) throws Exception {
-        node.createApplication(appName, deployHar, distributedState);
-        return null;
-    }
-
-    @Override
-    public String toString() {
-        return "Started application: " + appName;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java
deleted file mode 100644
index d0f9151..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.remote.ops;
-
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-public class JobCompleteNotifier implements RemoteOp<Void> {
-    private String nodeId;
-    private JobId jobId;
-    private JobStatus status;
-
-    public JobCompleteNotifier(String nodeId, JobId jobId, JobStatus status) {
-        this.nodeId = nodeId;
-        this.jobId = jobId;
-        this.status = status;
-    }
-
-    @Override
-    public Void execute(INodeController node) throws Exception {
-        node.cleanUpJoblet(jobId, status);
-        return null;
-    }
-
-    @Override
-    public String toString() {
-        return jobId + " Cleaning Up";
-    }
-
-    @Override
-    public String getNodeId() {
-        return nodeId;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
index 929462b..e6e6c2f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
@@ -49,6 +49,8 @@
 import edu.uci.ics.hyracks.control.cc.job.TaskClusterId;
 
 public class ActivityClusterPlanner {
+    private static final boolean USE_CONNECTOR_POLICY_IN_TASK_CLUSTER_CONSTRUCTION = true;
+
     private static final Logger LOGGER = Logger.getLogger(ActivityClusterPlanner.class.getName());
 
     private final JobScheduler scheduler;
@@ -64,13 +66,28 @@
         JobRun jobRun = scheduler.getJobRun();
         Map<ActivityId, ActivityPartitionDetails> pcMap = computePartitionCounts(ac);
 
+        Map<ActivityId, ActivityPlan> activityPlanMap = buildActivityPlanMap(ac, jobRun, pcMap);
+
+        assignConnectorPolicy(ac, activityPlanMap);
+
+        TaskCluster[] taskClusters = computeTaskClusters(ac, jobRun, activityPlanMap);
+
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Plan for " + ac);
+            LOGGER.info("Built " + taskClusters.length + " Task Clusters");
+            for (TaskCluster tc : taskClusters) {
+                LOGGER.info("Tasks: " + Arrays.toString(tc.getTasks()));
+            }
+        }
+
+        ac.setPlan(new ActivityClusterPlan(taskClusters, activityPlanMap));
+    }
+
+    private Map<ActivityId, ActivityPlan> buildActivityPlanMap(ActivityCluster ac, JobRun jobRun,
+            Map<ActivityId, ActivityPartitionDetails> pcMap) {
         Map<ActivityId, ActivityPlan> activityPlanMap = new HashMap<ActivityId, ActivityPlan>();
-        Set<ActivityId> activities = ac.getActivities();
-
-        Map<TaskId, Set<TaskId>> taskClusterMap = new HashMap<TaskId, Set<TaskId>>();
-
         Set<ActivityId> depAnIds = new HashSet<ActivityId>();
-        for (ActivityId anId : activities) {
+        for (ActivityId anId : ac.getActivities()) {
             depAnIds.clear();
             getDependencyActivityIds(depAnIds, anId);
             ActivityPartitionDetails apd = pcMap.get(anId);
@@ -94,53 +111,21 @@
                     tasks[i].getDependencies().add(dTaskId);
                     dTask.getDependents().add(tid);
                 }
-                Set<TaskId> cluster = new HashSet<TaskId>();
-                cluster.add(tid);
-                taskClusterMap.put(tid, cluster);
             }
             activityPlan.setTasks(tasks);
             activityPlanMap.put(anId, activityPlan);
         }
+        return activityPlanMap;
+    }
 
-        Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = assignConnectorPolicy(ac, activityPlanMap);
-        scheduler.getJobRun().getConnectorPolicyMap().putAll(connectorPolicies);
+    private TaskCluster[] computeTaskClusters(ActivityCluster ac, JobRun jobRun,
+            Map<ActivityId, ActivityPlan> activityPlanMap) {
+        Set<ActivityId> activities = ac.getActivities();
+        Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> taskConnectivity = computeTaskConnectivity(jobRun,
+                activityPlanMap, activities);
 
-        Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> taskConnectivity = new HashMap<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>>();
-        JobActivityGraph jag = jobRun.getJobActivityGraph();
-        BitSet targetBitmap = new BitSet();
-        for (ActivityId ac1 : activities) {
-            Task[] ac1TaskStates = activityPlanMap.get(ac1).getTasks();
-            int nProducers = ac1TaskStates.length;
-            List<IConnectorDescriptor> outputConns = jag.getActivityOutputConnectorDescriptors(ac1);
-            if (outputConns != null) {
-                for (IConnectorDescriptor c : outputConns) {
-                    ConnectorDescriptorId cdId = c.getConnectorId();
-                    ActivityId ac2 = jag.getConsumerActivity(cdId);
-                    Task[] ac2TaskStates = activityPlanMap.get(ac2).getTasks();
-                    int nConsumers = ac2TaskStates.length;
-                    for (int i = 0; i < nProducers; ++i) {
-                        c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
-                        List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = taskConnectivity.get(ac1TaskStates[i]
-                                .getTaskId());
-                        if (cInfoList == null) {
-                            cInfoList = new ArrayList<Pair<TaskId, ConnectorDescriptorId>>();
-                            taskConnectivity.put(ac1TaskStates[i].getTaskId(), cInfoList);
-                        }
-                        Set<TaskId> cluster = taskClusterMap.get(ac1TaskStates[i].getTaskId());
-                        for (int j = targetBitmap.nextSetBit(0); j >= 0; j = targetBitmap.nextSetBit(j + 1)) {
-                            TaskId targetTID = ac2TaskStates[j].getTaskId();
-                            cInfoList.add(Pair.<TaskId, ConnectorDescriptorId> of(targetTID, cdId));
-                            IConnectorPolicy cPolicy = connectorPolicies.get(cdId);
-                            if (cPolicy.requiresProducerConsumerCoscheduling()) {
-                                cluster.add(targetTID);
-                            }
-                        }
-                    }
-                }
-            }
-        }
-
-        TaskCluster[] taskClusters = buildTaskClusters(ac, activityPlanMap, taskClusterMap);
+        TaskCluster[] taskClusters = USE_CONNECTOR_POLICY_IN_TASK_CLUSTER_CONSTRUCTION ? buildConnectorPolicyAwareTaskClusters(
+                ac, activityPlanMap, taskConnectivity) : buildConnectorPolicyUnawareTaskClusters(ac, activityPlanMap);
 
         for (TaskCluster tc : taskClusters) {
             Set<TaskCluster> tcDependencyTaskClusters = tc.getDependencyTaskClusters();
@@ -170,20 +155,88 @@
                 }
             }
         }
+        return taskClusters;
+    }
 
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Plan for " + ac);
-            LOGGER.info("Built " + taskClusters.length + " Task Clusters");
-            for (TaskCluster tc : taskClusters) {
-                LOGGER.info("Tasks: " + Arrays.toString(tc.getTasks()));
+    private TaskCluster[] buildConnectorPolicyUnawareTaskClusters(ActivityCluster ac,
+            Map<ActivityId, ActivityPlan> activityPlanMap) {
+        List<Task> taskStates = new ArrayList<Task>();
+        for (ActivityId anId : ac.getActivities()) {
+            ActivityPlan ap = activityPlanMap.get(anId);
+            Task[] tasks = ap.getTasks();
+            for (Task t : tasks) {
+                taskStates.add(t);
+            }
+        }
+        TaskCluster tc = new TaskCluster(new TaskClusterId(ac.getActivityClusterId(), 0), ac,
+                taskStates.toArray(new Task[taskStates.size()]));
+        for (Task t : tc.getTasks()) {
+            t.setTaskCluster(tc);
+        }
+        return new TaskCluster[] { tc };
+    }
+
+    private Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> computeTaskConnectivity(JobRun jobRun,
+            Map<ActivityId, ActivityPlan> activityPlanMap, Set<ActivityId> activities) {
+        Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> taskConnectivity = new HashMap<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>>();
+        JobActivityGraph jag = jobRun.getJobActivityGraph();
+        BitSet targetBitmap = new BitSet();
+        for (ActivityId ac1 : activities) {
+            Task[] ac1TaskStates = activityPlanMap.get(ac1).getTasks();
+            int nProducers = ac1TaskStates.length;
+            List<IConnectorDescriptor> outputConns = jag.getActivityOutputConnectorDescriptors(ac1);
+            if (outputConns != null) {
+                for (IConnectorDescriptor c : outputConns) {
+                    ConnectorDescriptorId cdId = c.getConnectorId();
+                    ActivityId ac2 = jag.getConsumerActivity(cdId);
+                    Task[] ac2TaskStates = activityPlanMap.get(ac2).getTasks();
+                    int nConsumers = ac2TaskStates.length;
+                    for (int i = 0; i < nProducers; ++i) {
+                        c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
+                        List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = taskConnectivity.get(ac1TaskStates[i]
+                                .getTaskId());
+                        if (cInfoList == null) {
+                            cInfoList = new ArrayList<Pair<TaskId, ConnectorDescriptorId>>();
+                            taskConnectivity.put(ac1TaskStates[i].getTaskId(), cInfoList);
+                        }
+                        for (int j = targetBitmap.nextSetBit(0); j >= 0; j = targetBitmap.nextSetBit(j + 1)) {
+                            TaskId targetTID = ac2TaskStates[j].getTaskId();
+                            cInfoList.add(Pair.<TaskId, ConnectorDescriptorId> of(targetTID, cdId));
+                        }
+                    }
+                }
+            }
+        }
+        return taskConnectivity;
+    }
+
+    private TaskCluster[] buildConnectorPolicyAwareTaskClusters(ActivityCluster ac,
+            Map<ActivityId, ActivityPlan> activityPlanMap,
+            Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> taskConnectivity) {
+        Map<TaskId, Set<TaskId>> taskClusterMap = new HashMap<TaskId, Set<TaskId>>();
+        for (ActivityId anId : ac.getActivities()) {
+            ActivityPlan ap = activityPlanMap.get(anId);
+            Task[] tasks = ap.getTasks();
+            for (Task t : tasks) {
+                Set<TaskId> cluster = new HashSet<TaskId>();
+                TaskId tid = t.getTaskId();
+                cluster.add(tid);
+                taskClusterMap.put(tid, cluster);
             }
         }
 
-        ac.setPlan(new ActivityClusterPlan(taskClusters, activityPlanMap));
-    }
+        JobRun jobRun = ac.getJobRun();
+        Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = jobRun.getConnectorPolicyMap();
+        for (Map.Entry<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> e : taskConnectivity.entrySet()) {
+            Set<TaskId> cluster = taskClusterMap.get(e.getKey());
+            for (Pair<TaskId, ConnectorDescriptorId> p : e.getValue()) {
+                IConnectorPolicy cPolicy = connectorPolicies.get(p.getRight());
+                if (cPolicy.requiresProducerConsumerCoscheduling()) {
+                    cluster.add(p.getLeft());
+                }
+            }
+        }
 
-    private TaskCluster[] buildTaskClusters(ActivityCluster ac, Map<ActivityId, ActivityPlan> activityPlanMap,
-            Map<TaskId, Set<TaskId>> taskClusterMap) {
         /*
          * taskClusterMap contains for every TID x, x -> { coscheduled consumer TIDs U x }
          * We compute the transitive closure of this relation to find the largest set of
@@ -273,8 +326,7 @@
         }
     }
 
-    private Map<ConnectorDescriptorId, IConnectorPolicy> assignConnectorPolicy(ActivityCluster ac,
-            Map<ActivityId, ActivityPlan> taskMap) {
+    private void assignConnectorPolicy(ActivityCluster ac, Map<ActivityId, ActivityPlan> taskMap) {
         JobActivityGraph jag = scheduler.getJobRun().getJobActivityGraph();
         Map<ConnectorDescriptorId, IConnectorPolicy> cPolicyMap = new HashMap<ConnectorDescriptorId, IConnectorPolicy>();
         Set<ActivityId> activities = ac.getActivities();
@@ -300,7 +352,7 @@
                 }
             }
         }
-        return cPolicyMap;
+        ac.getJobRun().getConnectorPolicyMap().putAll(cPolicyMap);
     }
 
     private IConnectorPolicy assignConnectorPolicy(IConnectorDescriptor c, int nProducers, int nConsumers, int[] fanouts) {
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 e2c9ca3..9a443d9 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
@@ -184,8 +184,8 @@
     private void startRunnableActivityClusters() throws HyracksException {
         Set<TaskCluster> taskClusterRoots = new HashSet<TaskCluster>();
         findRunnableTaskClusterRoots(taskClusterRoots, rootActivityClusters);
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Runnable TC roots: " + taskClusterRoots + ", inProgressTaskClusters: "
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Runnable TC roots: " + taskClusterRoots + ", inProgressTaskClusters: "
                     + inProgressTaskClusters);
         }
         if (taskClusterRoots.isEmpty() && inProgressTaskClusters.isEmpty()) {
@@ -213,19 +213,19 @@
                 queue.add(new RankedRunnableTaskCluster(priority, tc));
             }
         }
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Ranked TCs: " + queue);
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Ranked TCs: " + queue);
         }
 
         Map<String, List<TaskAttemptDescriptor>> taskAttemptMap = new HashMap<String, List<TaskAttemptDescriptor>>();
         for (RankedRunnableTaskCluster rrtc : queue) {
             TaskCluster tc = rrtc.getTaskCluster();
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Found runnable TC: " + tc);
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Found runnable TC: " + tc);
                 List<TaskClusterAttempt> attempts = tc.getAttempts();
-                LOGGER.info("Attempts so far:" + attempts.size());
+                LOGGER.fine("Attempts so far:" + attempts.size());
                 for (TaskClusterAttempt tcAttempt : attempts) {
-                    LOGGER.info("Status: " + tcAttempt.getStatus());
+                    LOGGER.fine("Status: " + tcAttempt.getStatus());
                 }
             }
             assignTaskLocations(tc, taskAttemptMap);
@@ -245,16 +245,16 @@
      * Runnability(Non-schedulable TaskCluster) = {NOT_RUNNABLE, _} 
      */
     private Runnability assignRunnabilityRank(TaskCluster goal, Map<TaskCluster, Runnability> runnabilityMap) {
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Computing runnability: " + goal);
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Computing runnability: " + goal);
         }
         if (runnabilityMap.containsKey(goal)) {
             return runnabilityMap.get(goal);
         }
         TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(goal);
         if (lastAttempt != null) {
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Last Attempt Status: " + lastAttempt.getStatus());
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Last Attempt Status: " + lastAttempt.getStatus());
             }
             if (lastAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
                 Runnability runnability = new Runnability(Runnability.Tag.COMPLETED, Integer.MIN_VALUE);
@@ -271,15 +271,15 @@
         PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
         Runnability aggregateRunnability = new Runnability(Runnability.Tag.RUNNABLE, 0);
         for (PartitionId pid : goal.getRequiredPartitions()) {
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Inspecting required partition: " + pid);
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Inspecting required partition: " + pid);
             }
             Runnability runnability;
             ConnectorDescriptorId cdId = pid.getConnectorDescriptorId();
             IConnectorPolicy cPolicy = connectorPolicyMap.get(cdId);
             PartitionState maxState = pmm.getMaximumAvailableState(pid);
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Policy: " + cPolicy + " maxState: " + maxState);
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Policy: " + cPolicy + " maxState: " + maxState);
             }
             if (PartitionState.COMMITTED.equals(maxState)) {
                 runnability = new Runnability(Runnability.Tag.RUNNABLE, 0);
@@ -309,8 +309,12 @@
                 }
             }
             aggregateRunnability = Runnability.getWorstCase(aggregateRunnability, runnability);
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("aggregateRunnability: " + aggregateRunnability);
+            if (aggregateRunnability.getTag() == Runnability.Tag.NOT_RUNNABLE) {
+                // already not runnable -- cannot get better. bail.
+                break;
+            }
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("aggregateRunnability: " + aggregateRunnability);
             }
         }
         runnabilityMap.put(goal, aggregateRunnability);
@@ -470,14 +474,14 @@
     }
 
     private void abortTaskCluster(TaskClusterAttempt tcAttempt) {
-        LOGGER.info("Aborting task cluster: " + tcAttempt.getAttempt());
+        LOGGER.fine("Aborting task cluster: " + tcAttempt.getAttempt());
         Set<TaskAttemptId> abortTaskIds = new HashSet<TaskAttemptId>();
         Map<String, List<TaskAttemptId>> abortTaskAttemptMap = new HashMap<String, List<TaskAttemptId>>();
         for (TaskAttempt ta : tcAttempt.getTaskAttempts()) {
             TaskAttemptId taId = ta.getTaskAttemptId();
             TaskAttempt.TaskStatus status = ta.getStatus();
             abortTaskIds.add(taId);
-            LOGGER.info("Checking " + taId + ": " + ta.getStatus());
+            LOGGER.fine("Checking " + taId + ": " + ta.getStatus());
             if (status == TaskAttempt.TaskStatus.RUNNING || status == TaskAttempt.TaskStatus.COMPLETED) {
                 ta.setStatus(TaskAttempt.TaskStatus.ABORTED, null);
                 ta.setEndTime(System.currentTimeMillis());
@@ -490,12 +494,12 @@
             }
         }
         final JobId jobId = jobRun.getJobId();
-        LOGGER.info("Abort map for job: " + jobId + ": " + abortTaskAttemptMap);
+        LOGGER.fine("Abort map for job: " + jobId + ": " + abortTaskAttemptMap);
         for (Map.Entry<String, List<TaskAttemptId>> e : abortTaskAttemptMap.entrySet()) {
             final NodeControllerState node = ccs.getNodeMap().get(e.getKey());
             final List<TaskAttemptId> abortTaskAttempts = e.getValue();
             if (node != null) {
-                LOGGER.info("Aborting: " + abortTaskAttempts + " at " + e.getKey());
+                LOGGER.fine("Aborting: " + abortTaskAttempts + " at " + e.getKey());
                 ccs.getExecutor().execute(new Runnable() {
                     @Override
                     public void run() {
@@ -609,12 +613,12 @@
      */
     public void notifyTaskFailure(TaskAttempt ta, ActivityCluster ac, String details) {
         try {
-            LOGGER.info("Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
+            LOGGER.fine("Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
             TaskAttemptId taId = ta.getTaskAttemptId();
             TaskCluster tc = ta.getTask().getTaskCluster();
             TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
             if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
-                LOGGER.info("Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
+                LOGGER.fine("Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
                 ta.setStatus(TaskAttempt.TaskStatus.FAILED, details);
                 abortTaskCluster(lastAttempt);
                 lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.FAILED);
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java
index 3c8f7d0..15d6d1f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationCreateWork.java
@@ -20,34 +20,42 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
 
 public class ApplicationCreateWork extends AbstractWork {
     private final ClusterControllerService ccs;
     private final String appName;
-    private FutureValue<Object> fv;
+    private IResultCallback<Object> callback;
 
-    public ApplicationCreateWork(ClusterControllerService ccs, String appName, FutureValue<Object> fv) {
+    public ApplicationCreateWork(ClusterControllerService ccs, String appName, IResultCallback<Object> callback) {
         this.ccs = ccs;
         this.appName = appName;
-        this.fv = fv;
+        this.callback = callback;
     }
 
     @Override
     public void run() {
-        Map<String, CCApplicationContext> applications = ccs.getApplicationMap();
-        if (applications.containsKey(appName)) {
-            fv.setException(new HyracksException("Duplicate application with name: " + appName + " being created."));
-        }
-        CCApplicationContext appCtx;
         try {
-            appCtx = new CCApplicationContext(ccs.getServerContext(), ccs.getCCContext(), appName);
-        } catch (IOException e) {
-            fv.setException(e);
-            return;
+            Map<String, CCApplicationContext> applications = ccs.getApplicationMap();
+            if (applications.containsKey(appName)) {
+                callback.setException(new HyracksException("Duplicate application with name: " + appName
+                        + " being created."));
+                return;
+            }
+            CCApplicationContext appCtx;
+            try {
+                appCtx = new CCApplicationContext(ccs.getServerContext(), ccs.getCCContext(), appName);
+            } catch (IOException e) {
+                callback.setException(e);
+                return;
+            }
+            appCtx.setStatus(ApplicationStatus.CREATED);
+            applications.put(appName, appCtx);
+            callback.setValue(null);
+        } catch (Exception e) {
+            callback.setException(e);
         }
-        applications.put(appName, appCtx);
-        fv.setValue(null);
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
index c6af1b9..ac780c4 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationDestroyWork.java
@@ -14,62 +14,51 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.util.Map;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.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.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
 
 public class ApplicationDestroyWork extends AbstractWork {
     private final ClusterControllerService ccs;
     private final String appName;
-    private FutureValue<Object> fv;
+    private IResultCallback<Object> callback;
 
-    public ApplicationDestroyWork(ClusterControllerService ccs, String appName, FutureValue<Object> fv) {
+    public ApplicationDestroyWork(ClusterControllerService ccs, String appName, IResultCallback<Object> callback) {
         this.ccs = ccs;
         this.appName = appName;
-        this.fv = fv;
+        this.callback = callback;
     }
 
     @Override
     public void run() {
-        final ApplicationContext appCtx = ccs.getApplicationMap().remove(appName);
-        if (appCtx == null) {
-            fv.setException(new HyracksException("No application with name: " + appName));
-            return;
-        }
-        List<RemoteOp<Void>> opList = new ArrayList<RemoteOp<Void>>();
-        for (final String nodeId : ccs.getNodeMap().keySet()) {
-            opList.add(new ApplicationDestroyer(nodeId, appName));
-        }
-        final RemoteOp[] ops = opList.toArray(new RemoteOp[opList.size()]);
-        ccs.getExecutor().execute(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    RemoteRunner.runRemote(ccs, ops, null);
-                } catch (Exception e) {
-                    fv.setException(e);
-                    return;
-                }
-                ccs.getWorkQueue().schedule(new AbstractWork() {
-                    @Override
-                    public void run() {
-                        try {
-                            appCtx.deinitialize();
-                        } catch (Exception e) {
-                            fv.setException(e);
-                        }
-                        fv.setValue(null);
-                    }
-                });
+        try {
+            final CCApplicationContext appCtx = ccs.getApplicationMap().remove(appName);
+            if (appCtx == null) {
+                callback.setException(new HyracksException("No application with name: " + appName));
+                return;
             }
-        });
+            if (appCtx.getStatus() == ApplicationStatus.IN_DEINITIALIZATION
+                    || appCtx.getStatus() == ApplicationStatus.DEINITIALIZED) {
+                return;
+            }
+            Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
+            appCtx.getDeinitializationPendingNodeIds().addAll(nodeMap.keySet());
+            appCtx.setStatus(ApplicationStatus.IN_DEINITIALIZATION);
+            appCtx.setDeinitializationCallback(callback);
+            for (String nodeId : ccs.getNodeMap().keySet()) {
+                NodeControllerState nodeState = nodeMap.get(nodeId);
+                final INodeController node = nodeState.getNodeController();
+                node.destroyApplication(appName);
+            }
+        } catch (Exception e) {
+            callback.setException(e);
+        }
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
index 2022c7e..e4ad56c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStartWork.java
@@ -14,60 +14,67 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.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.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
 
 public class ApplicationStartWork extends AbstractWork {
     private final ClusterControllerService ccs;
     private final String appName;
-    private final FutureValue<Object> fv;
+    private final IResultCallback<Object> callback;
 
-    public ApplicationStartWork(ClusterControllerService ccs, String appName, FutureValue<Object> fv) {
+    public ApplicationStartWork(ClusterControllerService ccs, String appName, IResultCallback<Object> callback) {
         this.ccs = ccs;
         this.appName = appName;
-        this.fv = fv;
+        this.callback = callback;
     }
 
     @Override
     public void run() {
-        ApplicationContext appCtx = ccs.getApplicationMap().get(appName);
-        if (appCtx == null) {
-            fv.setException(new HyracksException("No application with name: " + appName));
-            return;
-        }
         try {
-            appCtx.initializeClassPath();
-            appCtx.initialize();
-            final byte[] distributedState = JavaSerializationUtils.serialize(appCtx.getDistributedState());
-            final boolean deployHar = appCtx.containsHar();
-            List<RemoteOp<Void>> opList = new ArrayList<RemoteOp<Void>>();
-            for (final String nodeId : ccs.getNodeMap().keySet()) {
-                opList.add(new ApplicationStarter(nodeId, appName, deployHar, distributedState));
+            final CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
+            if (appCtx == null) {
+                callback.setException(new HyracksException("No application with name: " + appName));
+                return;
             }
-            final RemoteOp[] ops = opList.toArray(new RemoteOp[opList.size()]);
+            if (appCtx.getStatus() != ApplicationStatus.CREATED) {
+                callback.setException(new HyracksException("Application in incorrect state for starting: "
+                        + appCtx.getStatus()));
+            }
+            final Map<String, NodeControllerState> nodeMapCopy = new HashMap<String, NodeControllerState>(
+                    ccs.getNodeMap());
+            appCtx.getInitializationPendingNodeIds().addAll(nodeMapCopy.keySet());
+            appCtx.setStatus(ApplicationStatus.IN_INITIALIZATION);
+            appCtx.setInitializationCallback(callback);
             ccs.getExecutor().execute(new Runnable() {
                 @Override
                 public void run() {
                     try {
-                        RemoteRunner.runRemote(ccs, ops, null);
-                        fv.setValue(null);
+                        appCtx.initializeClassPath();
+                        appCtx.initialize();
+                        final byte[] distributedState = JavaSerializationUtils.serialize(appCtx.getDistributedState());
+                        final boolean deployHar = appCtx.containsHar();
+                        for (final String nodeId : nodeMapCopy.keySet()) {
+                            NodeControllerState nodeState = nodeMapCopy.get(nodeId);
+                            final INodeController node = nodeState.getNodeController();
+                            node.createApplication(appName, deployHar, distributedState);
+                        }
                     } catch (Exception e) {
-                        fv.setException(e);
+                        callback.setException(e);
                     }
                 }
             });
         } catch (Exception e) {
-            fv.setException(e);
+            callback.setException(e);
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStateChangeWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStateChangeWork.java
new file mode 100644
index 0000000..f6271fe
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationStateChangeWork.java
@@ -0,0 +1,85 @@
+package edu.uci.ics.hyracks.control.cc.work;
+
+import java.util.Set;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
+import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+
+public class ApplicationStateChangeWork extends AbstractWork {
+    private static final Logger LOGGER = Logger.getLogger(ApplicationStateChangeWork.class.getName());
+
+    private final ClusterControllerService ccs;
+    private final CCNCFunctions.ApplicationStateChangeResponseFunction ascrf;
+
+    public ApplicationStateChangeWork(ClusterControllerService ccs,
+            CCNCFunctions.ApplicationStateChangeResponseFunction ascrf) {
+        this.ccs = ccs;
+        this.ascrf = ascrf;
+    }
+
+    @Override
+    public void run() {
+        final CCApplicationContext appCtx = ccs.getApplicationMap().get(ascrf.getApplicationName());
+        if (appCtx == null) {
+            LOGGER.warning("Got ApplicationStateChangeResponse for application " + ascrf.getApplicationName()
+                    + " that does not exist");
+            return;
+        }
+        switch (ascrf.getStatus()) {
+            case INITIALIZED: {
+                Set<String> pendingNodeIds = appCtx.getInitializationPendingNodeIds();
+                boolean changed = pendingNodeIds.remove(ascrf.getNodeId());
+                if (!changed) {
+                    LOGGER.warning("Got ApplicationStateChangeResponse for application " + ascrf.getApplicationName()
+                            + " from unexpected node " + ascrf.getNodeId() + " to state " + ascrf.getStatus());
+                    return;
+                }
+                if (pendingNodeIds.isEmpty()) {
+                    appCtx.setStatus(ApplicationStatus.INITIALIZED);
+                    IResultCallback<Object> callback = appCtx.getInitializationCallback();
+                    appCtx.setInitializationCallback(null);
+                    callback.setValue(null);
+                }
+                return;
+            }
+
+            case DEINITIALIZED: {
+                Set<String> pendingNodeIds = appCtx.getDeinitializationPendingNodeIds();
+                boolean changed = pendingNodeIds.remove(ascrf.getNodeId());
+                if (!changed) {
+                    LOGGER.warning("Got ApplicationStateChangeResponse for application " + ascrf.getApplicationName()
+                            + " from unexpected node " + ascrf.getNodeId() + " to state " + ascrf.getStatus());
+                    return;
+                }
+                if (pendingNodeIds.isEmpty()) {
+                    appCtx.setStatus(ApplicationStatus.DEINITIALIZED);
+                    ccs.getExecutor().execute(new Runnable() {
+                        @Override
+                        public void run() {
+                            try {
+                                appCtx.deinitialize();
+                            } catch (Exception e) {
+                                e.printStackTrace();
+                            }
+                            ccs.getWorkQueue().schedule(new AbstractWork() {
+                                @Override
+                                public void run() {
+                                    ccs.getApplicationMap().remove(ascrf.getApplicationName());
+                                    IResultCallback<Object> callback = appCtx.getDeinitializationCallback();
+                                    appCtx.setDeinitializationCallback(null);
+                                    callback.setValue(null);
+                                }
+                            });
+                        }
+                    });
+                }
+                return;
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusConditionVariableWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java
similarity index 60%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusConditionVariableWork.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java
index ff67928..1f29903 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusConditionVariableWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java
@@ -14,30 +14,23 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
-import edu.uci.ics.hyracks.api.job.JobId;
+import java.util.Map;
+import java.util.Set;
+
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
 import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class GetJobStatusConditionVariableWork extends SynchronizableWork {
+public class GetIpAddressNodeNameMapWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
-    private final JobId jobId;
-    private IJobStatusConditionVariable cVar;
+    private Map<String, Set<String>> map;
 
-    public GetJobStatusConditionVariableWork(ClusterControllerService ccs, JobId jobId) {
+    public GetIpAddressNodeNameMapWork(ClusterControllerService ccs, Map<String, Set<String>> map) {
         this.ccs = ccs;
-        this.jobId = jobId;
+        this.map = map;
     }
 
     @Override
     protected void doRun() throws Exception {
-        cVar = ccs.getActiveRunMap().get(jobId);
-        if (cVar == null) {
-            cVar = ccs.getRunMapArchive().get(jobId);
-        }
-    }
-
-    public IJobStatusConditionVariable getConditionVariable() {
-        return cVar;
+        map.putAll(ccs.getIpAddressNodeNameMap());
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusWork.java
index 7a8943a..c70e24f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobStatusWork.java
@@ -18,28 +18,31 @@
 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.common.work.IResultCallback;
 import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
 public class GetJobStatusWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final JobId jobId;
-    private JobStatus status;
+    private final IResultCallback<JobStatus> callback;
 
-    public GetJobStatusWork(ClusterControllerService ccs, JobId jobId) {
+    public GetJobStatusWork(ClusterControllerService ccs, JobId jobId, IResultCallback<JobStatus> callback) {
         this.ccs = ccs;
         this.jobId = jobId;
+        this.callback = callback;
     }
 
     @Override
     protected void doRun() throws Exception {
-        JobRun run = ccs.getActiveRunMap().get(jobId);
-        if (run == null) {
-            run = ccs.getRunMapArchive().get(jobId);
+        try {
+            JobRun run = ccs.getActiveRunMap().get(jobId);
+            if (run == null) {
+                run = ccs.getRunMapArchive().get(jobId);
+            }
+            JobStatus status = run == null ? null : run.getStatus();
+            callback.setValue(status);
+        } catch (Exception e) {
+            callback.setException(e);
         }
-        status = run == null ? null : run.getStatus();
-    }
-
-    public JobStatus getStatus() {
-        return status;
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
index 48d9b84..9e8d130 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
@@ -22,15 +22,16 @@
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
 
 public class GetNodeControllersInfoWork extends AbstractWork {
     private final ClusterControllerService ccs;
-    private FutureValue<Map<String, NodeControllerInfo>> fv;
+    private IResultCallback<Map<String, NodeControllerInfo>> callback;
 
-    public GetNodeControllersInfoWork(ClusterControllerService ccs, FutureValue<Map<String, NodeControllerInfo>> fv) {
+    public GetNodeControllersInfoWork(ClusterControllerService ccs,
+            IResultCallback<Map<String, NodeControllerInfo>> callback) {
         this.ccs = ccs;
-        this.fv = fv;
+        this.callback = callback;
     }
 
     @Override
@@ -38,9 +39,8 @@
         Map<String, NodeControllerInfo> result = new LinkedHashMap<String, NodeControllerInfo>();
         Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
         for (Map.Entry<String, NodeControllerState> e : nodeMap.entrySet()) {
-            result.put(e.getKey(), new NodeControllerInfo(e.getKey(), e.getValue().getDataPort().getIpAddress(),
-                    NodeStatus.ALIVE));
+            result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE));
         }
-        fv.setValue(result);
+        callback.setValue(result);
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
index e30a718..07088bb 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
@@ -20,9 +20,8 @@
 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.NodeControllerState;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
-import edu.uci.ics.hyracks.control.cc.remote.ops.JobCompleteNotifier;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
 public class JobCleanupWork extends AbstractWork {
@@ -54,22 +53,13 @@
         Set<String> targetNodes = run.getParticipatingNodeIds();
         run.getCleanupPendingNodeIds().addAll(targetNodes);
         run.setPendingStatus(status, exception);
-        final JobCompleteNotifier[] jcns = new JobCompleteNotifier[targetNodes.size()];
-        int i = 0;
         for (String n : targetNodes) {
-            jcns[i++] = new JobCompleteNotifier(n, jobId, status);
-        }
-        ccs.getExecutor().execute(new Runnable() {
-            @Override
-            public void run() {
-                if (jcns.length > 0) {
-                    try {
-                        RemoteRunner.runRemote(ccs, jcns, null);
-                    } catch (Exception e) {
-                        e.printStackTrace();
-                    }
-                }
+            NodeControllerState ncs = ccs.getNodeMap().get(n);
+            try {
+                ncs.getNodeController().cleanUpJoblet(jobId, status);
+            } catch (Exception e) {
+                e.printStackTrace();
             }
-        });
+        }
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java
index 14947cd..b7cf629 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java
@@ -30,6 +30,7 @@
 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.scheduler.JobScheduler;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
 import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
 public class JobCreateWork extends SynchronizableWork {
@@ -38,42 +39,50 @@
     private final EnumSet<JobFlag> jobFlags;
     private final JobId jobId;
     private final String appName;
+    private final IResultCallback<JobId> callback;
 
     public JobCreateWork(ClusterControllerService ccs, JobId jobId, String appName, byte[] jobSpec,
-            EnumSet<JobFlag> jobFlags) {
+            EnumSet<JobFlag> jobFlags, IResultCallback<JobId> callback) {
         this.jobId = jobId;
         this.ccs = ccs;
         this.jobSpec = jobSpec;
         this.jobFlags = jobFlags;
         this.appName = appName;
+        this.callback = callback;
     }
 
     @Override
     protected void doRun() throws Exception {
-        CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
-        if (appCtx == null) {
-            throw new HyracksException("No application with id " + appName + " found");
-        }
-        JobSpecification spec = appCtx.createJobSpecification(jobSpec);
-
-        final JobActivityGraphBuilder builder = new JobActivityGraphBuilder();
-        builder.init(appName, spec, jobFlags);
-        PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
-            @Override
-            public void visit(IOperatorDescriptor op) {
-                op.contributeActivities(builder);
+        try {
+            CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
+            if (appCtx == null) {
+                throw new HyracksException("No application with id " + appName + " found");
             }
-        });
-        final JobActivityGraph jag = builder.getActivityGraph();
+            JobSpecification spec = appCtx.createJobSpecification(jobSpec);
 
-        JobRun run = new JobRun(jobId, jag);
+            final JobActivityGraphBuilder builder = new JobActivityGraphBuilder();
+            builder.init(appName, spec, jobFlags);
+            PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
+                @Override
+                public void visit(IOperatorDescriptor op) {
+                    op.contributeActivities(builder);
+                }
+            });
+            final JobActivityGraph jag = builder.getActivityGraph();
 
-        run.setStatus(JobStatus.INITIALIZED, null);
+            JobRun run = new JobRun(jobId, jag);
 
-        ccs.getActiveRunMap().put(jobId, run);
-        JobScheduler jrs = new JobScheduler(ccs, run);
-        run.setScheduler(jrs);
-        appCtx.notifyJobCreation(jobId, spec);
+            run.setStatus(JobStatus.INITIALIZED, null);
+
+            ccs.getActiveRunMap().put(jobId, run);
+            JobScheduler jrs = new JobScheduler(ccs, run);
+            run.setScheduler(jrs);
+            appCtx.notifyJobCreation(jobId, spec);
+            callback.setValue(jobId);
+        } catch (Exception e) {
+            callback.setException(e);
+            return;
+        }
     }
 
     public JobId getJobId() {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
index 855e3e7..dd834ed 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
@@ -18,31 +18,39 @@
 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.common.work.IResultCallback;
 import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
 public class JobStartWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final JobId jobId;
+    private final IResultCallback<Object> callback;
 
-    public JobStartWork(ClusterControllerService ccs, JobId jobId) {
+    public JobStartWork(ClusterControllerService ccs, JobId jobId, IResultCallback<Object> callback) {
         this.ccs = ccs;
         this.jobId = jobId;
+        this.callback = callback;
     }
 
     @Override
     protected void doRun() throws Exception {
-        JobRun run = ccs.getActiveRunMap().get(jobId);
-        if (run == null) {
-            throw new Exception("Unable to find job with id = " + jobId);
-        }
-        if (run.getStatus() != JobStatus.INITIALIZED) {
-            throw new Exception("Job already started");
-        }
-        run.setStatus(JobStatus.RUNNING, null);
         try {
-            run.getScheduler().startJob();
+            JobRun run = ccs.getActiveRunMap().get(jobId);
+            if (run == null) {
+                throw new Exception("Unable to find job with id = " + jobId);
+            }
+            if (run.getStatus() != JobStatus.INITIALIZED) {
+                throw new Exception("Job already started");
+            }
+            run.setStatus(JobStatus.RUNNING, null);
+            try {
+                run.getScheduler().startJob();
+            } catch (Exception e) {
+                ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, run.getJobId(), JobStatus.FAILURE, e));
+            }
+            callback.setValue(null);
         } catch (Exception e) {
-            ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, run.getJobId(), JobStatus.FAILURE, e));
+            callback.setException(e);
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
index 15daff5..e4e0aae 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
@@ -14,29 +14,65 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
+import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
+import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions;
+import edu.uci.ics.hyracks.control.common.ipc.NodeControllerRemoteProxy;
 import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
 
 public class RegisterNodeWork extends SynchronizableWork {
-    private final ClusterControllerService ccs;
-    private final String nodeId;
-    private final NodeControllerState state;
+    private static final Logger LOGGER = Logger.getLogger(RegisterNodeWork.class.getName());
 
-    public RegisterNodeWork(ClusterControllerService ccs, String nodeId, NodeControllerState state) {
+    private final ClusterControllerService ccs;
+    private final NodeRegistration reg;
+
+    public RegisterNodeWork(ClusterControllerService ccs, NodeRegistration reg) {
         this.ccs = ccs;
-        this.nodeId = nodeId;
-        this.state = state;
+        this.reg = reg;
     }
 
     @Override
     protected void doRun() throws Exception {
-        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
-        if (nodeMap.containsKey(nodeId)) {
-            throw new Exception("Node with this name already registered.");
+        String id = reg.getNodeId();
+
+        IIPCHandle ncIPCHandle = ccs.getClusterIPC().getHandle(reg.getNodeControllerAddress());
+        CCNCFunctions.NodeRegistrationResult result = null;
+        try {
+            INodeController nodeController = new NodeControllerRemoteProxy(ncIPCHandle);
+
+            NodeControllerState state = new NodeControllerState(nodeController, reg);
+            Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
+            if (nodeMap.containsKey(id)) {
+                throw new Exception("Node with this name already registered.");
+            }
+            nodeMap.put(id, state);
+            Map<String, Set<String>> ipAddressNodeNameMap = ccs.getIpAddressNodeNameMap();
+            String ipAddress = state.getNCConfig().dataIPAddress;
+            Set<String> nodes = ipAddressNodeNameMap.get(ipAddress);
+            if (nodes == null) {
+                nodes = new HashSet<String>();
+                ipAddressNodeNameMap.put(ipAddress, nodes);
+            }
+            nodes.add(id);
+            LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
+            NodeParameters params = new NodeParameters();
+            params.setClusterControllerInfo(ccs.getClusterControllerInfo());
+            params.setHeartbeatPeriod(ccs.getCCConfig().heartbeatPeriod);
+            params.setProfileDumpPeriod(ccs.getCCConfig().profileDumpPeriod);
+            result = new CCNCFunctions.NodeRegistrationResult(params, null);
+        } catch (Exception e) {
+            result = new CCNCFunctions.NodeRegistrationResult(null, e);
         }
-        nodeMap.put(nodeId, state);
+        ncIPCHandle.send(-1, result, null);
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/WaitForJobCompletionWork.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/WaitForJobCompletionWork.java
new file mode 100644
index 0000000..6cfe025
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/WaitForJobCompletionWork.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.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.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class WaitForJobCompletionWork extends SynchronizableWork {
+    private final ClusterControllerService ccs;
+    private final JobId jobId;
+    private final IResultCallback<Object> callback;
+
+    public WaitForJobCompletionWork(ClusterControllerService ccs, JobId jobId, IResultCallback<Object> callback) {
+        this.ccs = ccs;
+        this.jobId = jobId;
+        this.callback = callback;
+    }
+
+    @Override
+    protected void doRun() throws Exception {
+        final IJobStatusConditionVariable cRunningVar = ccs.getActiveRunMap().get(jobId);
+        if (cRunningVar != null) {
+            ccs.getExecutor().execute(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        cRunningVar.waitForCompletion();
+                        callback.setValue(null);
+                    } catch (Exception e) {
+                        callback.setException(e);
+                    }
+                }
+            });
+        } else {
+            final IJobStatusConditionVariable cArchivedVar = ccs.getRunMapArchive().get(jobId);
+            ccs.getExecutor().execute(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        cArchivedVar.waitForCompletion();
+                        callback.setValue(null);
+                    } catch (Exception e) {
+                        callback.setException(e);
+                    }
+                }
+            });
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java
index 42d0ecb..21e3e9d 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/AbstractRemoteService.java
@@ -19,4 +19,4 @@
 public abstract class AbstractRemoteService implements IService {
     public AbstractRemoteService() {
     }
-}
+}
\ No newline at end of file
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 d2a06be..9f29fbd 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
@@ -59,7 +59,6 @@
         this.serverCtx = serverCtx;
         this.appName = appName;
         this.applicationRootDir = new File(new File(serverCtx.getBaseDir(), APPLICATION_ROOT), appName);
-        status = ApplicationStatus.CREATED;
         FileUtils.deleteDirectory(applicationRootDir);
         applicationRootDir.mkdirs();
     }
@@ -87,9 +86,6 @@
     }
 
     public void initialize() throws Exception {
-        if (status != ApplicationStatus.CREATED) {
-            throw new IllegalStateException();
-        }
         if (deploymentDescriptor != null) {
             String bootstrapClass = null;
             switch (serverCtx.getServerType()) {
@@ -107,7 +103,6 @@
                 start();
             }
         }
-        status = ApplicationStatus.INITIALIZED;
     }
 
     protected abstract void start() throws Exception;
@@ -168,7 +163,6 @@
     }
 
     public void deinitialize() throws Exception {
-        status = ApplicationStatus.DEINITIALIZED;
         stop();
         File expandedFolder = getExpandedFolder();
         FileUtils.deleteDirectory(expandedFolder);
@@ -203,4 +197,12 @@
     public ClassLoader getClassLoader() {
         return classLoader;
     }
-}
+
+    public void setStatus(ApplicationStatus status) {
+        this.status = status;
+    }
+
+    public ApplicationStatus getStatus() {
+        return status;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationStatus.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationStatus.java
index 8a97ad6..b45c4f1 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationStatus.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationStatus.java
@@ -16,6 +16,8 @@
 
 public enum ApplicationStatus {
     CREATED,
+    IN_INITIALIZATION,
     INITIALIZED,
+    IN_DEINITIALIZATION,
     DEINITIALIZED
 }
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
index 40e9347..a03f5c9 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
@@ -18,7 +18,7 @@
 
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
 import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
 import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
@@ -27,7 +27,7 @@
 import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
 
 public interface IClusterController {
-    public NodeParameters registerNode(NodeRegistration reg) throws Exception;
+    public void registerNode(NodeRegistration reg) throws Exception;
 
     public void unregisterNode(String nodeId) throws Exception;
 
@@ -45,4 +45,6 @@
     public void registerPartitionProvider(PartitionDescriptor partitionDescriptor) throws Exception;
 
     public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception;
+
+    public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception;
 }
\ 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 c55f34a..574f552 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
@@ -52,6 +52,9 @@
     @Option(name = "-dcache-client-path", usage = "Sets the path to store the files retrieved from the DCache server (default /tmp/dcache-client)")
     public String dcacheClientPath = "/tmp/dcache-client";
 
+    @Option(name = "-net-thread-count", usage = "Number of threads to use for Network I/O (default: 1)")
+    public int nNetThreads = 1;
+
     public void toCommandLine(List<String> cList) {
         cList.add("-cc-host");
         cList.add(ccHost);
@@ -75,5 +78,7 @@
         }
         cList.add("-dcache-client-path");
         cList.add(dcacheClientPath);
+        cList.add("-net-thread-count");
+        cList.add(String.valueOf(nNetThreads));
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
index cd9086f..8ca0fa4 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
@@ -33,4 +33,8 @@
     public double systemLoadAverage;
     public long[] gcCollectionCounts;
     public long[] gcCollectionTimes;
+    public long netPayloadBytesRead;
+    public long netPayloadBytesWritten;
+    public long netSignalingBytesRead;
+    public long netSignalingBytesWritten;
 }
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
new file mode 100644
index 0000000..9ea4636
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
@@ -0,0 +1,792 @@
+/*
+ * 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.common.ipc;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+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.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
+import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
+import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
+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.job.PartitionState;
+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.TaskProfile;
+import edu.uci.ics.hyracks.ipc.api.IPayloadSerializerDeserializer;
+import edu.uci.ics.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
+
+public class CCNCFunctions {
+    private static final int FID_CODE_SIZE = 1;
+
+    public enum FunctionId {
+        REGISTER_NODE,
+        UNREGISTER_NODE,
+        NOTIFY_JOBLET_CLEANUP,
+        NOTIFY_TASK_COMPLETE,
+        NOTIFY_TASK_FAILURE,
+        NODE_HEARTBEAT,
+        REPORT_PROFILE,
+        REGISTER_PARTITION_PROVIDER,
+        REGISTER_PARTITION_REQUEST,
+        APPLICATION_STATE_CHANGE_RESPONSE,
+
+        NODE_REGISTRATION_RESULT,
+        START_TASKS,
+        ABORT_TASKS,
+        CLEANUP_JOBLET,
+        CREATE_APPLICATION,
+        DESTROY_APPLICATION,
+        REPORT_PARTITION_AVAILABILITY,
+
+        OTHER
+    }
+
+    public static abstract class Function implements Serializable {
+        private static final long serialVersionUID = 1L;
+
+        public abstract FunctionId getFunctionId();
+    }
+
+    public static class RegisterNodeFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final NodeRegistration reg;
+
+        public RegisterNodeFunction(NodeRegistration reg) {
+            this.reg = reg;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.REGISTER_NODE;
+        }
+
+        public NodeRegistration getNodeRegistration() {
+            return reg;
+        }
+    }
+
+    public static class UnregisterNodeFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final String nodeId;
+
+        public UnregisterNodeFunction(String nodeId) {
+            this.nodeId = nodeId;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.UNREGISTER_NODE;
+        }
+
+        public String getNodeId() {
+            return nodeId;
+        }
+    }
+
+    public static class NotifyTaskCompleteFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+        private final TaskAttemptId taskId;
+        private final String nodeId;
+        private final TaskProfile statistics;
+
+        public NotifyTaskCompleteFunction(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics) {
+            this.jobId = jobId;
+            this.taskId = taskId;
+            this.nodeId = nodeId;
+            this.statistics = statistics;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.NOTIFY_TASK_COMPLETE;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public TaskAttemptId getTaskId() {
+            return taskId;
+        }
+
+        public String getNodeId() {
+            return nodeId;
+        }
+
+        public TaskProfile getStatistics() {
+            return statistics;
+        }
+    }
+
+    public static class NotifyTaskFailureFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+        private final TaskAttemptId taskId;
+        private final String nodeId;
+        private final String details;
+
+        public NotifyTaskFailureFunction(JobId jobId, TaskAttemptId taskId, String nodeId, String details) {
+            this.jobId = jobId;
+            this.taskId = taskId;
+            this.nodeId = nodeId;
+            this.details = details;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.NOTIFY_TASK_FAILURE;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public TaskAttemptId getTaskId() {
+            return taskId;
+        }
+
+        public String getNodeId() {
+            return nodeId;
+        }
+
+        public String getDetails() {
+            return details;
+        }
+    }
+
+    public static class NotifyJobletCleanupFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+        private final String nodeId;
+
+        public NotifyJobletCleanupFunction(JobId jobId, String nodeId) {
+            this.jobId = jobId;
+            this.nodeId = nodeId;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.NOTIFY_JOBLET_CLEANUP;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public String getNodeId() {
+            return nodeId;
+        }
+    }
+
+    public static class NodeHeartbeatFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final String nodeId;
+        private final HeartbeatData hbData;
+
+        public NodeHeartbeatFunction(String nodeId, HeartbeatData hbData) {
+            this.nodeId = nodeId;
+            this.hbData = hbData;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.NODE_HEARTBEAT;
+        }
+
+        public String getNodeId() {
+            return nodeId;
+        }
+
+        public HeartbeatData getHeartbeatData() {
+            return hbData;
+        }
+    }
+
+    public static class ReportProfileFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final String nodeId;
+        private final List<JobProfile> profiles;
+
+        public ReportProfileFunction(String nodeId, List<JobProfile> profiles) {
+            this.nodeId = nodeId;
+            this.profiles = profiles;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.REPORT_PROFILE;
+        }
+
+        public String getNodeId() {
+            return nodeId;
+        }
+
+        public List<JobProfile> getProfiles() {
+            return profiles;
+        }
+    }
+
+    public static class RegisterPartitionProviderFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final PartitionDescriptor partitionDescriptor;
+
+        public RegisterPartitionProviderFunction(PartitionDescriptor partitionDescriptor) {
+            this.partitionDescriptor = partitionDescriptor;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.REGISTER_PARTITION_PROVIDER;
+        }
+
+        public PartitionDescriptor getPartitionDescriptor() {
+            return partitionDescriptor;
+        }
+
+        public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
+            ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
+            DataInputStream dis = new DataInputStream(bais);
+
+            // Read PartitionId
+            PartitionId pid = readPartitionId(dis);
+
+            // Read nodeId
+            String nodeId = dis.readUTF();
+
+            // Read TaskAttemptId
+            TaskAttemptId taId = readTaskAttemptId(dis);
+
+            // Read reusable flag
+            boolean reusable = dis.readBoolean();
+
+            // Read Partition State
+            PartitionState state = readPartitionState(dis);
+
+            PartitionDescriptor pd = new PartitionDescriptor(pid, nodeId, taId, reusable);
+            pd.setState(state);
+            return new RegisterPartitionProviderFunction(pd);
+        }
+
+        public static void serialize(OutputStream out, Object object) throws Exception {
+            RegisterPartitionProviderFunction fn = (RegisterPartitionProviderFunction) object;
+
+            DataOutputStream dos = new DataOutputStream(out);
+
+            PartitionDescriptor pd = fn.getPartitionDescriptor();
+
+            // Write PartitionId
+            writePartitionId(dos, pd.getPartitionId());
+
+            // Write nodeId
+            dos.writeUTF(pd.getNodeId());
+
+            // Write TaskAttemptId
+            writeTaskAttemptId(dos, pd.getProducingTaskAttemptId());
+
+            // Write reusable flag
+            dos.writeBoolean(pd.isReusable());
+
+            // Write Partition State
+            writePartitionState(dos, pd.getState());
+        }
+    }
+
+    public static class RegisterPartitionRequestFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final PartitionRequest partitionRequest;
+
+        public RegisterPartitionRequestFunction(PartitionRequest partitionRequest) {
+            this.partitionRequest = partitionRequest;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.REGISTER_PARTITION_REQUEST;
+        }
+
+        public PartitionRequest getPartitionRequest() {
+            return partitionRequest;
+        }
+
+        public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
+            ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
+            DataInputStream dis = new DataInputStream(bais);
+
+            // Read PartitionId
+            PartitionId pid = readPartitionId(dis);
+
+            // Read nodeId
+            String nodeId = dis.readUTF();
+
+            // Read TaskAttemptId
+            TaskAttemptId taId = readTaskAttemptId(dis);
+
+            // Read Partition State
+            PartitionState state = readPartitionState(dis);
+
+            PartitionRequest pr = new PartitionRequest(pid, nodeId, taId, state);
+            return new RegisterPartitionRequestFunction(pr);
+        }
+
+        public static void serialize(OutputStream out, Object object) throws Exception {
+            RegisterPartitionRequestFunction fn = (RegisterPartitionRequestFunction) object;
+
+            DataOutputStream dos = new DataOutputStream(out);
+
+            PartitionRequest pr = fn.getPartitionRequest();
+
+            // Write PartitionId
+            writePartitionId(dos, pr.getPartitionId());
+
+            // Write nodeId
+            dos.writeUTF(pr.getNodeId());
+
+            // Write TaskAttemptId
+            writeTaskAttemptId(dos, pr.getRequestingTaskAttemptId());
+
+            // Write Partition State
+            writePartitionState(dos, pr.getMinimumState());
+        }
+    }
+
+    public static class ApplicationStateChangeResponseFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final String nodeId;
+        private final String appName;
+        private final ApplicationStatus status;
+
+        public ApplicationStateChangeResponseFunction(String nodeId, String appName, ApplicationStatus status) {
+            this.nodeId = nodeId;
+            this.appName = appName;
+            this.status = status;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.APPLICATION_STATE_CHANGE_RESPONSE;
+        }
+
+        public String getNodeId() {
+            return nodeId;
+        }
+
+        public String getApplicationName() {
+            return appName;
+        }
+
+        public ApplicationStatus getStatus() {
+            return status;
+        }
+    }
+
+    public static class NodeRegistrationResult extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final NodeParameters params;
+
+        private final Exception exception;
+
+        public NodeRegistrationResult(NodeParameters params, Exception exception) {
+            this.params = params;
+            this.exception = exception;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.NODE_REGISTRATION_RESULT;
+        }
+
+        public NodeParameters getNodeParameters() {
+            return params;
+        }
+
+        public Exception getException() {
+            return exception;
+        }
+    }
+
+    public static class StartTasksFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final String appName;
+        private final JobId jobId;
+        private final byte[] planBytes;
+        private final List<TaskAttemptDescriptor> taskDescriptors;
+        private final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies;
+
+        public StartTasksFunction(String appName, JobId jobId, byte[] planBytes,
+                List<TaskAttemptDescriptor> taskDescriptors,
+                Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies) {
+            this.appName = appName;
+            this.jobId = jobId;
+            this.planBytes = planBytes;
+            this.taskDescriptors = taskDescriptors;
+            this.connectorPolicies = connectorPolicies;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.START_TASKS;
+        }
+
+        public String getAppName() {
+            return appName;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public byte[] getPlanBytes() {
+            return planBytes;
+        }
+
+        public List<TaskAttemptDescriptor> getTaskDescriptors() {
+            return taskDescriptors;
+        }
+
+        public Map<ConnectorDescriptorId, IConnectorPolicy> getConnectorPolicies() {
+            return connectorPolicies;
+        }
+    }
+
+    public static class AbortTasksFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+        private final List<TaskAttemptId> tasks;
+
+        public AbortTasksFunction(JobId jobId, List<TaskAttemptId> tasks) {
+            this.jobId = jobId;
+            this.tasks = tasks;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.ABORT_TASKS;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public List<TaskAttemptId> getTasks() {
+            return tasks;
+        }
+    }
+
+    public static class CleanupJobletFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final JobId jobId;
+        private final JobStatus status;
+
+        public CleanupJobletFunction(JobId jobId, JobStatus status) {
+            this.jobId = jobId;
+            this.status = status;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.CLEANUP_JOBLET;
+        }
+
+        public JobId getJobId() {
+            return jobId;
+        }
+
+        public JobStatus getStatus() {
+            return status;
+        }
+    }
+
+    public static class CreateApplicationFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final String appName;
+        private final boolean deployHar;
+        private final byte[] serializedDistributedState;
+
+        public CreateApplicationFunction(String appName, boolean deployHar, byte[] serializedDistributedState) {
+            this.appName = appName;
+            this.deployHar = deployHar;
+            this.serializedDistributedState = serializedDistributedState;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.CREATE_APPLICATION;
+        }
+
+        public String getAppName() {
+            return appName;
+        }
+
+        public boolean isDeployHar() {
+            return deployHar;
+        }
+
+        public byte[] getSerializedDistributedState() {
+            return serializedDistributedState;
+        }
+    }
+
+    public static class DestroyApplicationFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final String appName;
+
+        public DestroyApplicationFunction(String appName) {
+            this.appName = appName;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.DESTROY_APPLICATION;
+        }
+
+        public String getAppName() {
+            return appName;
+        }
+    }
+
+    public static class ReportPartitionAvailabilityFunction extends Function {
+        private static final long serialVersionUID = 1L;
+
+        private final PartitionId pid;
+        private final NetworkAddress networkAddress;
+
+        public ReportPartitionAvailabilityFunction(PartitionId pid, NetworkAddress networkAddress) {
+            this.pid = pid;
+            this.networkAddress = networkAddress;
+        }
+
+        @Override
+        public FunctionId getFunctionId() {
+            return FunctionId.REPORT_PARTITION_AVAILABILITY;
+        }
+
+        public PartitionId getPartitionId() {
+            return pid;
+        }
+
+        public NetworkAddress getNetworkAddress() {
+            return networkAddress;
+        }
+
+        public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
+            ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
+            DataInputStream dis = new DataInputStream(bais);
+
+            // Read PartitionId
+            PartitionId pid = readPartitionId(dis);
+
+            // Read NetworkAddress
+            NetworkAddress networkAddress = readNetworkAddress(dis);
+
+            return new ReportPartitionAvailabilityFunction(pid, networkAddress);
+        }
+
+        public static void serialize(OutputStream out, Object object) throws Exception {
+            ReportPartitionAvailabilityFunction fn = (ReportPartitionAvailabilityFunction) object;
+
+            DataOutputStream dos = new DataOutputStream(out);
+
+            // Write PartitionId
+            writePartitionId(dos, fn.getPartitionId());
+
+            // Write NetworkAddress
+            writeNetworkAddress(dos, fn.getNetworkAddress());
+        }
+    }
+
+    public static class SerializerDeserializer implements IPayloadSerializerDeserializer {
+        private final JavaSerializationBasedPayloadSerializerDeserializer javaSerde;
+
+        public SerializerDeserializer() {
+            javaSerde = new JavaSerializationBasedPayloadSerializerDeserializer();
+        }
+
+        @Override
+        public Object deserializeObject(ByteBuffer buffer, int length) throws Exception {
+            if (length < FID_CODE_SIZE) {
+                throw new IllegalStateException("Message size too small: " + length);
+            }
+            byte fid = buffer.get();
+            return deserialize(fid, buffer, length - FID_CODE_SIZE);
+        }
+
+        @Override
+        public Exception deserializeException(ByteBuffer buffer, int length) throws Exception {
+            if (length < FID_CODE_SIZE) {
+                throw new IllegalStateException("Message size too small: " + length);
+            }
+            byte fid = buffer.get();
+            if (fid != FunctionId.OTHER.ordinal()) {
+                throw new IllegalStateException("Expected FID for OTHER, found: " + fid);
+            }
+            return (Exception) deserialize(fid, buffer, length - FID_CODE_SIZE);
+        }
+
+        @Override
+        public byte[] serializeObject(Object object) throws Exception {
+            if (object instanceof Function) {
+                Function fn = (Function) object;
+                return serialize(object, (byte) fn.getFunctionId().ordinal());
+            } else {
+                return serialize(object, (byte) FunctionId.OTHER.ordinal());
+            }
+        }
+
+        @Override
+        public byte[] serializeException(Exception object) throws Exception {
+            return serialize(object, (byte) FunctionId.OTHER.ordinal());
+        }
+
+        private byte[] serialize(Object object, byte fid) throws Exception {
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            baos.write(fid);
+            serialize(baos, object, fid);
+            JavaSerializationBasedPayloadSerializerDeserializer.serialize(baos, object);
+            baos.close();
+            return baos.toByteArray();
+        }
+
+        private void serialize(OutputStream out, Object object, byte fid) throws Exception {
+            switch (FunctionId.values()[fid]) {
+                case REGISTER_PARTITION_PROVIDER:
+                    RegisterPartitionProviderFunction.serialize(out, object);
+                    return;
+
+                case REGISTER_PARTITION_REQUEST:
+                    RegisterPartitionRequestFunction.serialize(out, object);
+                    return;
+
+                case REPORT_PARTITION_AVAILABILITY:
+                    ReportPartitionAvailabilityFunction.serialize(out, object);
+                    return;
+            }
+            JavaSerializationBasedPayloadSerializerDeserializer.serialize(out, object);
+        }
+
+        private Object deserialize(byte fid, ByteBuffer buffer, int length) throws Exception {
+            switch (FunctionId.values()[fid]) {
+                case REGISTER_PARTITION_PROVIDER:
+                    return RegisterPartitionProviderFunction.deserialize(buffer, length);
+
+                case REGISTER_PARTITION_REQUEST:
+                    return RegisterPartitionRequestFunction.deserialize(buffer, length);
+
+                case REPORT_PARTITION_AVAILABILITY:
+                    return ReportPartitionAvailabilityFunction.deserialize(buffer, length);
+            }
+
+            return javaSerde.deserializeObject(buffer, length);
+        }
+    }
+
+    private static PartitionId readPartitionId(DataInputStream dis) throws IOException {
+        long jobId = dis.readLong();
+        int cdid = dis.readInt();
+        int senderIndex = dis.readInt();
+        int receiverIndex = dis.readInt();
+        PartitionId pid = new PartitionId(new JobId(jobId), new ConnectorDescriptorId(cdid), senderIndex, receiverIndex);
+        return pid;
+    }
+
+    private static void writePartitionId(DataOutputStream dos, PartitionId pid) throws IOException {
+        dos.writeLong(pid.getJobId().getId());
+        dos.writeInt(pid.getConnectorDescriptorId().getId());
+        dos.writeInt(pid.getSenderIndex());
+        dos.writeInt(pid.getReceiverIndex());
+    }
+
+    private static TaskAttemptId readTaskAttemptId(DataInputStream dis) throws IOException {
+        int odid = dis.readInt();
+        int aid = dis.readInt();
+        int partition = dis.readInt();
+        int attempt = dis.readInt();
+        TaskAttemptId taId = new TaskAttemptId(new TaskId(new ActivityId(new OperatorDescriptorId(odid), aid),
+                partition), attempt);
+        return taId;
+    }
+
+    private static void writeTaskAttemptId(DataOutputStream dos, TaskAttemptId taId) throws IOException {
+        TaskId tid = taId.getTaskId();
+        ActivityId aid = tid.getActivityId();
+        OperatorDescriptorId odId = aid.getOperatorDescriptorId();
+        dos.writeInt(odId.getId());
+        dos.writeInt(aid.getLocalId());
+        dos.writeInt(tid.getPartition());
+        dos.writeInt(taId.getAttempt());
+    }
+
+    private static PartitionState readPartitionState(DataInputStream dis) throws IOException {
+        PartitionState state = PartitionState.values()[dis.readInt()];
+        return state;
+    }
+
+    private static void writePartitionState(DataOutputStream dos, PartitionState state) throws IOException {
+        dos.writeInt(state.ordinal());
+    }
+
+    private static NetworkAddress readNetworkAddress(DataInputStream dis) throws IOException {
+        int bLen = dis.readInt();
+        byte[] ipAddress = new byte[bLen];
+        dis.read(ipAddress);
+        int port = dis.readInt();
+        NetworkAddress networkAddress = new NetworkAddress(ipAddress, port);
+        return networkAddress;
+    }
+
+    private static void writeNetworkAddress(DataOutputStream dos, NetworkAddress networkAddress) throws IOException {
+        byte[] ipAddress = networkAddress.getIpAddress();
+        dos.writeInt(ipAddress.length);
+        dos.write(ipAddress);
+        dos.writeInt(networkAddress.getPort());
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerDelegateIPCI.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerDelegateIPCI.java
deleted file mode 100644
index e35f962..0000000
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerDelegateIPCI.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.common.ipc;
-
-import edu.uci.ics.hyracks.control.common.base.IClusterController;
-import edu.uci.ics.hyracks.control.common.ipc.ClusterControllerFunctions.Function;
-import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.IIPCI;
-
-public class ClusterControllerDelegateIPCI implements IIPCI {
-    private final IClusterController cc;
-
-    public ClusterControllerDelegateIPCI(IClusterController cc) {
-        this.cc = cc;
-    }
-
-    @Override
-    public Object call(IIPCHandle caller, Object req) throws Exception {
-        ClusterControllerFunctions.Function fn = (Function) req;
-        switch (fn.getFunctionId()) {
-            case REGISTER_NODE: {
-                ClusterControllerFunctions.RegisterNodeFunction rnf = (ClusterControllerFunctions.RegisterNodeFunction) fn;
-                return cc.registerNode(rnf.getNodeRegistration());
-            }
-
-            case UNREGISTER_NODE: {
-                ClusterControllerFunctions.UnregisterNodeFunction unf = (ClusterControllerFunctions.UnregisterNodeFunction) fn;
-                cc.unregisterNode(unf.getNodeId());
-                return null;
-            }
-
-            case NODE_HEARTBEAT: {
-                ClusterControllerFunctions.NodeHeartbeatFunction nhf = (ClusterControllerFunctions.NodeHeartbeatFunction) fn;
-                cc.nodeHeartbeat(nhf.getNodeId(), nhf.getHeartbeatData());
-                return null;
-            }
-
-            case NOTIFY_JOBLET_CLEANUP: {
-                ClusterControllerFunctions.NotifyJobletCleanupFunction njcf = (ClusterControllerFunctions.NotifyJobletCleanupFunction) fn;
-                cc.notifyJobletCleanup(njcf.getJobId(), njcf.getNodeId());
-                return null;
-            }
-
-            case REPORT_PROFILE: {
-                ClusterControllerFunctions.ReportProfileFunction rpf = (ClusterControllerFunctions.ReportProfileFunction) fn;
-                cc.reportProfile(rpf.getNodeId(), rpf.getProfiles());
-                return null;
-            }
-
-            case NOTIFY_TASK_COMPLETE: {
-                ClusterControllerFunctions.NotifyTaskCompleteFunction ntcf = (ClusterControllerFunctions.NotifyTaskCompleteFunction) fn;
-                cc.notifyTaskComplete(ntcf.getJobId(), ntcf.getTaskId(), ntcf.getNodeId(), ntcf.getStatistics());
-                return null;
-            }
-            case NOTIFY_TASK_FAILURE: {
-                ClusterControllerFunctions.NotifyTaskFailureFunction ntff = (ClusterControllerFunctions.NotifyTaskFailureFunction) fn;
-                cc.notifyTaskFailure(ntff.getJobId(), ntff.getTaskId(), ntff.getDetails(), ntff.getDetails());
-                return null;
-            }
-
-            case REGISTER_PARTITION_PROVIDER: {
-                ClusterControllerFunctions.RegisterPartitionProviderFunction rppf = (ClusterControllerFunctions.RegisterPartitionProviderFunction) fn;
-                cc.registerPartitionProvider(rppf.getPartitionDescriptor());
-                return null;
-            }
-
-            case REGISTER_PARTITION_REQUEST: {
-                ClusterControllerFunctions.RegisterPartitionRequestFunction rprf = (ClusterControllerFunctions.RegisterPartitionRequestFunction) fn;
-                cc.registerPartitionRequest(rprf.getPartitionRequest());
-                return null;
-            }
-        }
-        throw new IllegalArgumentException("Unknown function: " + fn.getFunctionId());
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerFunctions.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerFunctions.java
deleted file mode 100644
index 4c76357..0000000
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerFunctions.java
+++ /dev/null
@@ -1,272 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.common.ipc;
-
-import java.io.Serializable;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
-import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
-import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
-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;
-
-public class ClusterControllerFunctions {
-    public enum FunctionId {
-        REGISTER_NODE,
-        UNREGISTER_NODE,
-        NOTIFY_TASK_COMPLETE,
-        NOTIFY_TASK_FAILURE,
-        NOTIFY_JOBLET_CLEANUP,
-        NODE_HEARTBEAT,
-        REPORT_PROFILE,
-        REGISTER_PARTITION_PROVIDER,
-        REGISTER_PARTITION_REQUEST,
-    }
-
-    public static abstract class Function implements Serializable {
-        private static final long serialVersionUID = 1L;
-
-        public abstract FunctionId getFunctionId();
-    }
-
-    public static class RegisterNodeFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final NodeRegistration reg;
-
-        public RegisterNodeFunction(NodeRegistration reg) {
-            this.reg = reg;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.REGISTER_NODE;
-        }
-
-        public NodeRegistration getNodeRegistration() {
-            return reg;
-        }
-    }
-
-    public static class UnregisterNodeFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final String nodeId;
-
-        public UnregisterNodeFunction(String nodeId) {
-            this.nodeId = nodeId;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.UNREGISTER_NODE;
-        }
-
-        public String getNodeId() {
-            return nodeId;
-        }
-    }
-
-    public static class NotifyTaskCompleteFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final JobId jobId;
-        private final TaskAttemptId taskId;
-        private final String nodeId;
-        private final TaskProfile statistics;
-
-        public NotifyTaskCompleteFunction(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics) {
-            this.jobId = jobId;
-            this.taskId = taskId;
-            this.nodeId = nodeId;
-            this.statistics = statistics;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.NOTIFY_TASK_COMPLETE;
-        }
-
-        public JobId getJobId() {
-            return jobId;
-        }
-
-        public TaskAttemptId getTaskId() {
-            return taskId;
-        }
-
-        public String getNodeId() {
-            return nodeId;
-        }
-
-        public TaskProfile getStatistics() {
-            return statistics;
-        }
-    }
-
-    public static class NotifyTaskFailureFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final JobId jobId;
-        private final TaskAttemptId taskId;
-        private final String nodeId;
-        private final String details;
-
-        public NotifyTaskFailureFunction(JobId jobId, TaskAttemptId taskId, String nodeId, String details) {
-            this.jobId = jobId;
-            this.taskId = taskId;
-            this.nodeId = nodeId;
-            this.details = details;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.NOTIFY_TASK_FAILURE;
-        }
-
-        public JobId getJobId() {
-            return jobId;
-        }
-
-        public TaskAttemptId getTaskId() {
-            return taskId;
-        }
-
-        public String getNodeId() {
-            return nodeId;
-        }
-
-        public String getDetails() {
-            return details;
-        }
-    }
-
-    public static class NotifyJobletCleanupFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final JobId jobId;
-        private final String nodeId;
-
-        public NotifyJobletCleanupFunction(JobId jobId, String nodeId) {
-            this.jobId = jobId;
-            this.nodeId = nodeId;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.NOTIFY_JOBLET_CLEANUP;
-        }
-
-        public JobId getJobId() {
-            return jobId;
-        }
-
-        public String getNodeId() {
-            return nodeId;
-        }
-    }
-
-    public static class NodeHeartbeatFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final String nodeId;
-        private final HeartbeatData hbData;
-
-        public NodeHeartbeatFunction(String nodeId, HeartbeatData hbData) {
-            this.nodeId = nodeId;
-            this.hbData = hbData;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.NODE_HEARTBEAT;
-        }
-
-        public String getNodeId() {
-            return nodeId;
-        }
-
-        public HeartbeatData getHeartbeatData() {
-            return hbData;
-        }
-    }
-
-    public static class ReportProfileFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final String nodeId;
-        private final List<JobProfile> profiles;
-
-        public ReportProfileFunction(String nodeId, List<JobProfile> profiles) {
-            this.nodeId = nodeId;
-            this.profiles = profiles;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.REPORT_PROFILE;
-        }
-
-        public String getNodeId() {
-            return nodeId;
-        }
-
-        public List<JobProfile> getProfiles() {
-            return profiles;
-        }
-    }
-
-    public static class RegisterPartitionProviderFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final PartitionDescriptor partitionDescriptor;
-
-        public RegisterPartitionProviderFunction(PartitionDescriptor partitionDescriptor) {
-            this.partitionDescriptor = partitionDescriptor;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.REGISTER_PARTITION_PROVIDER;
-        }
-
-        public PartitionDescriptor getPartitionDescriptor() {
-            return partitionDescriptor;
-        }
-    }
-
-    public static class RegisterPartitionRequestFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final PartitionRequest partitionRequest;
-
-        public RegisterPartitionRequestFunction(PartitionRequest partitionRequest) {
-            this.partitionRequest = partitionRequest;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.REGISTER_PARTITION_REQUEST;
-        }
-
-        public PartitionRequest getPartitionRequest() {
-            return partitionRequest;
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
index 0aeab72..a0dabdd 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
@@ -18,8 +18,8 @@
 
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
 import edu.uci.ics.hyracks.control.common.base.IClusterController;
-import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
 import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
 import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
@@ -27,7 +27,6 @@
 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.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.SyncRMI;
 
 public class ClusterControllerRemoteProxy implements IClusterController {
     private final IIPCHandle ipcHandle;
@@ -37,75 +36,72 @@
     }
 
     @Override
-    public NodeParameters registerNode(NodeRegistration reg) throws Exception {
-        SyncRMI sync = new SyncRMI();
-        ClusterControllerFunctions.RegisterNodeFunction fn = new ClusterControllerFunctions.RegisterNodeFunction(reg);
-        NodeParameters result = (NodeParameters) sync.call(ipcHandle, fn);
-        return result;
+    public void registerNode(NodeRegistration reg) throws Exception {
+        CCNCFunctions.RegisterNodeFunction fn = new CCNCFunctions.RegisterNodeFunction(reg);
+        ipcHandle.send(-1, fn, null);
     }
 
     @Override
     public void unregisterNode(String nodeId) throws Exception {
-        SyncRMI sync = new SyncRMI();
-        ClusterControllerFunctions.UnregisterNodeFunction fn = new ClusterControllerFunctions.UnregisterNodeFunction(
+        CCNCFunctions.UnregisterNodeFunction fn = new CCNCFunctions.UnregisterNodeFunction(
                 nodeId);
-        sync.call(ipcHandle, fn);
+        ipcHandle.send(-1, fn, null);
     }
 
     @Override
     public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
             throws Exception {
-        SyncRMI sync = new SyncRMI();
-        ClusterControllerFunctions.NotifyTaskCompleteFunction fn = new ClusterControllerFunctions.NotifyTaskCompleteFunction(
+        CCNCFunctions.NotifyTaskCompleteFunction fn = new CCNCFunctions.NotifyTaskCompleteFunction(
                 jobId, taskId, nodeId, statistics);
-        sync.call(ipcHandle, fn);
+        ipcHandle.send(-1, fn, null);
     }
 
     @Override
     public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, String details) throws Exception {
-        SyncRMI sync = new SyncRMI();
-        ClusterControllerFunctions.NotifyTaskFailureFunction fn = new ClusterControllerFunctions.NotifyTaskFailureFunction(
+        CCNCFunctions.NotifyTaskFailureFunction fn = new CCNCFunctions.NotifyTaskFailureFunction(
                 jobId, taskId, nodeId, details);
-        sync.call(ipcHandle, fn);
+        ipcHandle.send(-1, fn, null);
     }
 
     @Override
     public void notifyJobletCleanup(JobId jobId, String nodeId) throws Exception {
-        SyncRMI sync = new SyncRMI();
-        ClusterControllerFunctions.NotifyJobletCleanupFunction fn = new ClusterControllerFunctions.NotifyJobletCleanupFunction(
+        CCNCFunctions.NotifyJobletCleanupFunction fn = new CCNCFunctions.NotifyJobletCleanupFunction(
                 jobId, nodeId);
-        sync.call(ipcHandle, fn);
+        ipcHandle.send(-1, fn, null);
     }
 
     @Override
     public void nodeHeartbeat(String id, HeartbeatData hbData) throws Exception {
-        SyncRMI sync = new SyncRMI();
-        ClusterControllerFunctions.NodeHeartbeatFunction fn = new ClusterControllerFunctions.NodeHeartbeatFunction(id,
+        CCNCFunctions.NodeHeartbeatFunction fn = new CCNCFunctions.NodeHeartbeatFunction(id,
                 hbData);
-        sync.call(ipcHandle, fn);
+        ipcHandle.send(-1, fn, null);
     }
 
     @Override
     public void reportProfile(String id, List<JobProfile> profiles) throws Exception {
-        SyncRMI sync = new SyncRMI();
-        ClusterControllerFunctions.ReportProfileFunction fn = new ClusterControllerFunctions.ReportProfileFunction(id,
+        CCNCFunctions.ReportProfileFunction fn = new CCNCFunctions.ReportProfileFunction(id,
                 profiles);
-        sync.call(ipcHandle, fn);
+        ipcHandle.send(-1, fn, null);
     }
 
     @Override
     public void registerPartitionProvider(PartitionDescriptor partitionDescriptor) throws Exception {
-        SyncRMI sync = new SyncRMI();
-        ClusterControllerFunctions.RegisterPartitionProviderFunction fn = new ClusterControllerFunctions.RegisterPartitionProviderFunction(
+        CCNCFunctions.RegisterPartitionProviderFunction fn = new CCNCFunctions.RegisterPartitionProviderFunction(
                 partitionDescriptor);
-        sync.call(ipcHandle, fn);
+        ipcHandle.send(-1, fn, null);
     }
 
     @Override
     public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception {
-        SyncRMI sync = new SyncRMI();
-        ClusterControllerFunctions.RegisterPartitionRequestFunction fn = new ClusterControllerFunctions.RegisterPartitionRequestFunction(
+        CCNCFunctions.RegisterPartitionRequestFunction fn = new CCNCFunctions.RegisterPartitionRequestFunction(
                 partitionRequest);
-        sync.call(ipcHandle, fn);
+        ipcHandle.send(-1, fn, null);
+    }
+
+    @Override
+    public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception {
+        CCNCFunctions.ApplicationStateChangeResponseFunction fn = new CCNCFunctions.ApplicationStateChangeResponseFunction(
+                nodeId, appName, status);
+        ipcHandle.send(-1, fn, null);
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerDelegateIPCI.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerDelegateIPCI.java
deleted file mode 100644
index f3e51ec..0000000
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerDelegateIPCI.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.common.ipc;
-
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.IIPCI;
-
-public class NodeControllerDelegateIPCI implements IIPCI {
-    private final INodeController nc;
-
-    public NodeControllerDelegateIPCI(INodeController nc) {
-        this.nc = nc;
-    }
-
-    @Override
-    public Object call(IIPCHandle caller, Object req) throws Exception {
-        NodeControllerFunctions.Function fn = (NodeControllerFunctions.Function) req;
-        switch (fn.getFunctionId()) {
-            case START_TASKS: {
-                NodeControllerFunctions.StartTasksFunction stf = (NodeControllerFunctions.StartTasksFunction) fn;
-                nc.startTasks(stf.getAppName(), stf.getJobId(), stf.getPlanBytes(), stf.getTaskDescriptors(),
-                        stf.getConnectorPolicies());
-                return null;
-            }
-
-            case ABORT_TASKS: {
-                NodeControllerFunctions.AbortTasksFunction atf = (NodeControllerFunctions.AbortTasksFunction) fn;
-                nc.abortTasks(atf.getJobId(), atf.getTasks());
-                return null;
-            }
-
-            case CLEANUP_JOBLET: {
-                NodeControllerFunctions.CleanupJobletFunction cjf = (NodeControllerFunctions.CleanupJobletFunction) fn;
-                nc.cleanUpJoblet(cjf.getJobId(), cjf.getStatus());
-                return null;
-            }
-
-            case CREATE_APPLICATION: {
-                NodeControllerFunctions.CreateApplicationFunction caf = (NodeControllerFunctions.CreateApplicationFunction) fn;
-                nc.createApplication(caf.getAppName(), caf.isDeployHar(), caf.getSerializedDistributedState());
-                return null;
-            }
-
-            case DESTROY_APPLICATION: {
-                NodeControllerFunctions.DestroyApplicationFunction daf = (NodeControllerFunctions.DestroyApplicationFunction) fn;
-                nc.destroyApplication(daf.getAppName());
-                return null;
-            }
-
-            case REPORT_PARTITION_AVAILABILITY: {
-                NodeControllerFunctions.ReportPartitionAvailabilityFunction rpaf = (NodeControllerFunctions.ReportPartitionAvailabilityFunction) fn;
-                nc.reportPartitionAvailability(rpaf.getPartitionId(), rpaf.getNetworkAddress());
-                return null;
-            }
-        }
-        throw new IllegalArgumentException("Unknown function: " + fn.getFunctionId());
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerFunctions.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerFunctions.java
deleted file mode 100644
index 0d39c1b..0000000
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerFunctions.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.common.ipc;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.comm.NetworkAddress;
-import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
-import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
-
-public class NodeControllerFunctions {
-    public enum FunctionId {
-        START_TASKS,
-        ABORT_TASKS,
-        CLEANUP_JOBLET,
-        CREATE_APPLICATION,
-        DESTROY_APPLICATION,
-        REPORT_PARTITION_AVAILABILITY
-    }
-
-    public static abstract class Function implements Serializable {
-        private static final long serialVersionUID = 1L;
-
-        public abstract FunctionId getFunctionId();
-    }
-
-    public static class StartTasksFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final String appName;
-        private final JobId jobId;
-        private final byte[] planBytes;
-        private final List<TaskAttemptDescriptor> taskDescriptors;
-        private final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies;
-
-        public StartTasksFunction(String appName, JobId jobId, byte[] planBytes,
-                List<TaskAttemptDescriptor> taskDescriptors,
-                Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies) {
-            this.appName = appName;
-            this.jobId = jobId;
-            this.planBytes = planBytes;
-            this.taskDescriptors = taskDescriptors;
-            this.connectorPolicies = connectorPolicies;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.START_TASKS;
-        }
-
-        public String getAppName() {
-            return appName;
-        }
-
-        public JobId getJobId() {
-            return jobId;
-        }
-
-        public byte[] getPlanBytes() {
-            return planBytes;
-        }
-
-        public List<TaskAttemptDescriptor> getTaskDescriptors() {
-            return taskDescriptors;
-        }
-
-        public Map<ConnectorDescriptorId, IConnectorPolicy> getConnectorPolicies() {
-            return connectorPolicies;
-        }
-    }
-
-    public static class AbortTasksFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final JobId jobId;
-        private final List<TaskAttemptId> tasks;
-
-        public AbortTasksFunction(JobId jobId, List<TaskAttemptId> tasks) {
-            this.jobId = jobId;
-            this.tasks = tasks;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.ABORT_TASKS;
-        }
-
-        public JobId getJobId() {
-            return jobId;
-        }
-
-        public List<TaskAttemptId> getTasks() {
-            return tasks;
-        }
-    }
-
-    public static class CleanupJobletFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final JobId jobId;
-        private final JobStatus status;
-
-        public CleanupJobletFunction(JobId jobId, JobStatus status) {
-            this.jobId = jobId;
-            this.status = status;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.CLEANUP_JOBLET;
-        }
-
-        public JobId getJobId() {
-            return jobId;
-        }
-
-        public JobStatus getStatus() {
-            return status;
-        }
-    }
-
-    public static class CreateApplicationFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final String appName;
-        private final boolean deployHar;
-        private final byte[] serializedDistributedState;
-
-        public CreateApplicationFunction(String appName, boolean deployHar, byte[] serializedDistributedState) {
-            this.appName = appName;
-            this.deployHar = deployHar;
-            this.serializedDistributedState = serializedDistributedState;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.CREATE_APPLICATION;
-        }
-
-        public String getAppName() {
-            return appName;
-        }
-
-        public boolean isDeployHar() {
-            return deployHar;
-        }
-
-        public byte[] getSerializedDistributedState() {
-            return serializedDistributedState;
-        }
-    }
-
-    public static class DestroyApplicationFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final String appName;
-
-        public DestroyApplicationFunction(String appName) {
-            this.appName = appName;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.DESTROY_APPLICATION;
-        }
-
-        public String getAppName() {
-            return appName;
-        }
-    }
-
-    public static class ReportPartitionAvailabilityFunction extends Function {
-        private static final long serialVersionUID = 1L;
-
-        private final PartitionId pid;
-        private final NetworkAddress networkAddress;
-
-        public ReportPartitionAvailabilityFunction(PartitionId pid, NetworkAddress networkAddress) {
-            this.pid = pid;
-            this.networkAddress = networkAddress;
-        }
-
-        @Override
-        public FunctionId getFunctionId() {
-            return FunctionId.REPORT_PARTITION_AVAILABILITY;
-        }
-
-        public PartitionId getPartitionId() {
-            return pid;
-        }
-
-        public NetworkAddress getNetworkAddress() {
-            return networkAddress;
-        }
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
index ccd9468..961a9e3 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
@@ -27,7 +27,6 @@
 import edu.uci.ics.hyracks.control.common.base.INodeController;
 import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
 import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.SyncRMI;
 
 public class NodeControllerRemoteProxy implements INodeController {
     private final IIPCHandle ipcHandle;
@@ -39,48 +38,43 @@
     @Override
     public void startTasks(String appName, JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
             Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies) throws Exception {
-        SyncRMI sync = new SyncRMI();
-        NodeControllerFunctions.StartTasksFunction stf = new NodeControllerFunctions.StartTasksFunction(appName, jobId,
+        CCNCFunctions.StartTasksFunction stf = new CCNCFunctions.StartTasksFunction(appName, jobId,
                 planBytes, taskDescriptors, connectorPolicies);
-        sync.call(ipcHandle, stf);
+        ipcHandle.send(-1, stf, null);
     }
 
     @Override
     public void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception {
-        SyncRMI sync = new SyncRMI();
-        NodeControllerFunctions.AbortTasksFunction atf = new NodeControllerFunctions.AbortTasksFunction(jobId, tasks);
-        sync.call(ipcHandle, atf);
+        CCNCFunctions.AbortTasksFunction atf = new CCNCFunctions.AbortTasksFunction(jobId, tasks);
+        ipcHandle.send(-1, atf, null);
     }
 
     @Override
     public void cleanUpJoblet(JobId jobId, JobStatus status) throws Exception {
-        SyncRMI sync = new SyncRMI();
-        NodeControllerFunctions.CleanupJobletFunction cjf = new NodeControllerFunctions.CleanupJobletFunction(jobId,
+        CCNCFunctions.CleanupJobletFunction cjf = new CCNCFunctions.CleanupJobletFunction(jobId,
                 status);
-        sync.call(ipcHandle, cjf);
+        ipcHandle.send(-1, cjf, null);
     }
 
     @Override
     public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
             throws Exception {
-        SyncRMI sync = new SyncRMI();
-        NodeControllerFunctions.CreateApplicationFunction caf = new NodeControllerFunctions.CreateApplicationFunction(
+        CCNCFunctions.CreateApplicationFunction caf = new CCNCFunctions.CreateApplicationFunction(
                 appName, deployHar, serializedDistributedState);
-        sync.call(ipcHandle, caf);
+        ipcHandle.send(-1, caf, null);
     }
 
     @Override
     public void destroyApplication(String appName) throws Exception {
-        SyncRMI sync = new SyncRMI();
-        NodeControllerFunctions.DestroyApplicationFunction daf = new NodeControllerFunctions.DestroyApplicationFunction(
+        CCNCFunctions.DestroyApplicationFunction daf = new CCNCFunctions.DestroyApplicationFunction(
                 appName);
-        sync.call(ipcHandle, daf);
+        ipcHandle.send(-1, daf, null);
     }
 
     @Override
     public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception {
-        NodeControllerFunctions.ReportPartitionAvailabilityFunction rpaf = new NodeControllerFunctions.ReportPartitionAvailabilityFunction(
+        CCNCFunctions.ReportPartitionAvailabilityFunction rpaf = new CCNCFunctions.ReportPartitionAvailabilityFunction(
                 pid, networkAddress);
-        ipcHandle.send(rpaf, null);
+        ipcHandle.send(-1, rpaf, null);
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/FutureValue.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/FutureValue.java
index 00565b6..7eb4ff6 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/FutureValue.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/FutureValue.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.common.work;
 
-public class FutureValue<T> {
+public class FutureValue<T> implements IResultCallback<T> {
     private boolean done;
 
     private T value;
@@ -27,6 +27,7 @@
         e = null;
     }
 
+    @Override
     public synchronized void setValue(T value) {
         done = true;
         this.value = value;
@@ -34,6 +35,7 @@
         notifyAll();
     }
 
+    @Override
     public synchronized void setException(Exception e) {
         done = true;
         this.e = e;
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/IPCResponder.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/IPCResponder.java
new file mode 100644
index 0000000..dcea864
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/IPCResponder.java
@@ -0,0 +1,33 @@
+package edu.uci.ics.hyracks.control.common.work;
+
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.exceptions.IPCException;
+
+public class IPCResponder<T> implements IResultCallback<T> {
+    private final IIPCHandle handle;
+
+    private final long rmid;
+
+    public IPCResponder(IIPCHandle handle, long rmid) {
+        this.handle = handle;
+        this.rmid = rmid;
+    }
+
+    @Override
+    public void setValue(T result) {
+        try {
+            handle.send(rmid, result, null);
+        } catch (IPCException e) {
+            e.printStackTrace();
+        }
+    }
+
+    @Override
+    public void setException(Exception e) {
+        try {
+            handle.send(rmid, null, e);
+        } catch (IPCException e1) {
+            e1.printStackTrace();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/IResultCallback.java
similarity index 78%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
copy to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/IResultCallback.java
index c728b0b..80c3d76 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/IResultCallback.java
@@ -12,10 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.control.common.work;
 
-public interface Accumulator<T, R> {
-    public void accumulate(T o);
+public interface IResultCallback<T> {
+    public void setValue(T result);
 
-    public R getResult();
+    public void setException(Exception e);
 }
\ No newline at end of file
diff --git a/hyracks-control-nc/pom.xml b/hyracks-control-nc/pom.xml
index 64a409c..8c56958 100644
--- a/hyracks-control-nc/pom.xml
+++ b/hyracks-control-nc/pom.xml
@@ -1,9 +1,6 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
   <modelVersion>4.0.0</modelVersion>
-  <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-control-nc</artifactId>
-  <version>0.2.0-SNAPSHOT</version>
-
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>hyracks</artifactId>
@@ -37,6 +34,11 @@
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-net</artifactId>
+  		<version>0.2.0-SNAPSHOT</version>
+  	</dependency>
   </dependencies>
   <reporting>
     <plugins>
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 d869515..0f512a7 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
@@ -39,33 +39,25 @@
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
-import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
 import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.api.partitions.PartitionId;
 import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
 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;
 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.controllers.NodeRegistration;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatSchema;
+import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions;
 import edu.uci.ics.hyracks.control.common.ipc.ClusterControllerRemoteProxy;
-import edu.uci.ics.hyracks.control.common.ipc.NodeControllerDelegateIPCI;
-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.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.NetworkManager;
 import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
 import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
 import edu.uci.ics.hyracks.control.nc.work.AbortTasksWork;
@@ -76,9 +68,11 @@
 import edu.uci.ics.hyracks.control.nc.work.ReportPartitionAvailabilityWork;
 import edu.uci.ics.hyracks.control.nc.work.StartTasksWork;
 import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.IIPCI;
 import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.PerformanceCounters;
 
-public class NodeControllerService extends AbstractRemoteService implements INodeController {
+public class NodeControllerService extends AbstractRemoteService {
     private static Logger LOGGER = Logger.getLogger(NodeControllerService.class.getName());
 
     private NCConfig ncConfig;
@@ -91,12 +85,16 @@
 
     private final PartitionManager partitionManager;
 
-    private final ConnectionManager connectionManager;
+    private final NetworkManager netManager;
 
     private final WorkQueue queue;
 
     private final Timer timer;
 
+    private boolean registrationPending;
+
+    private Exception registrationException;
+
     private IClusterController ccs;
 
     private final Map<JobId, Joblet> jobletMap;
@@ -125,15 +123,15 @@
         this.ncConfig = ncConfig;
         id = ncConfig.nodeId;
         executor = Executors.newCachedThreadPool();
-        NodeControllerDelegateIPCI ipci = new NodeControllerDelegateIPCI(this);
-        ipc = new IPCSystem(new InetSocketAddress(ncConfig.clusterNetIPAddress, 0), ipci, executor);
+        NodeControllerIPCI ipci = new NodeControllerIPCI();
+        ipc = new IPCSystem(new InetSocketAddress(ncConfig.clusterNetIPAddress, 0), ipci,
+                new CCNCFunctions.SerializerDeserializer());
         this.ctx = new RootHyracksContext(ncConfig.frameSize, new IOManager(getDevices(ncConfig.ioDevices), executor));
         if (id == null) {
             throw new Exception("id not set");
         }
-        connectionManager = new ConnectionManager(ctx, getIpAddress(ncConfig));
         partitionManager = new PartitionManager(this);
-        connectionManager.setPartitionRequestListener(partitionManager);
+        netManager = new NetworkManager(ctx, getIpAddress(ncConfig), partitionManager, ncConfig.nNetThreads);
 
         queue = new WorkQueue();
         jobletMap = new Hashtable<JobId, Joblet>();
@@ -146,6 +144,7 @@
         threadMXBean = ManagementFactory.getThreadMXBean();
         runtimeMXBean = ManagementFactory.getRuntimeMXBean();
         osMXBean = ManagementFactory.getOperatingSystemMXBean();
+        registrationPending = true;
     }
 
     public IHyracksRootContext getRootContext() {
@@ -162,11 +161,18 @@
         return devices;
     }
 
+    private synchronized void setNodeRegistrationResult(NodeParameters parameters, Exception exception) {
+        this.nodeParameters = parameters;
+        this.registrationException = exception;
+        this.registrationPending = false;
+        notifyAll();
+    }
+
     @Override
     public void start() throws Exception {
         LOGGER.log(Level.INFO, "Starting NodeControllerService");
         ipc.start();
-        connectionManager.start();
+        netManager.start();
         IIPCHandle ccIPCHandle = ipc.getHandle(new InetSocketAddress(ncConfig.ccHost, ncConfig.ccPort));
         this.ccs = new ClusterControllerRemoteProxy(ccIPCHandle);
         HeartbeatSchema.GarbageCollectorInfo[] gcInfos = new HeartbeatSchema.GarbageCollectorInfo[gcMXBeans.size()];
@@ -174,9 +180,19 @@
             gcInfos[i] = new HeartbeatSchema.GarbageCollectorInfo(gcMXBeans.get(i).getName());
         }
         HeartbeatSchema hbSchema = new HeartbeatSchema(gcInfos);
-        this.nodeParameters = ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig,
-                connectionManager.getNetworkAddress(), osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(),
-                osMXBean.getAvailableProcessors(), hbSchema));
+        ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig, netManager.getNetworkAddress(),
+                osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), osMXBean.getAvailableProcessors(),
+                hbSchema));
+
+        synchronized (this) {
+            while (registrationPending) {
+                wait();
+            }
+        }
+        if (registrationException != null) {
+            throw registrationException;
+        }
+
         queue.start();
 
         heartbeatTask = new HeartbeatTask(ccs);
@@ -197,7 +213,7 @@
         LOGGER.log(Level.INFO, "Stopping NodeControllerService");
         partitionManager.close();
         heartbeatTask.cancel();
-        connectionManager.stop();
+        netManager.stop();
         queue.stop();
         LOGGER.log(Level.INFO, "Stopped NodeControllerService");
     }
@@ -218,8 +234,8 @@
         return jobletMap;
     }
 
-    public ConnectionManager getConnectionManager() {
-        return connectionManager;
+    public NetworkManager getNetworkManager() {
+        return netManager;
     }
 
     public PartitionManager getPartitionManager() {
@@ -238,53 +254,10 @@
         return executor;
     }
 
-    @Override
-    public void startTasks(String appName, final JobId jobId, byte[] jagBytes,
-            List<TaskAttemptDescriptor> taskDescriptors,
-            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap) throws Exception {
-        StartTasksWork stw = new StartTasksWork(this, appName, jobId, jagBytes, taskDescriptors, connectorPoliciesMap);
-        queue.schedule(stw);
-    }
-
-    @Override
-    public void cleanUpJoblet(JobId jobId, JobStatus status) throws Exception {
-        CleanupJobletWork cjw = new CleanupJobletWork(this, jobId, status);
-        queue.schedule(cjw);
-    }
-
     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.schedule(atw);
-    }
-
-    @Override
-    public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
-            throws Exception {
-        FutureValue<Object> fv = new FutureValue<Object>();
-        CreateApplicationWork caw = new CreateApplicationWork(this, appName, deployHar, serializedDistributedState, fv);
-        queue.schedule(caw);
-        fv.get();
-    }
-
-    @Override
-    public void destroyApplication(String appName) throws Exception {
-        FutureValue<Object> fv = new FutureValue<Object>();
-        DestroyApplicationWork daw = new DestroyApplicationWork(this, appName, fv);
-        queue.schedule(daw);
-        fv.get();
-    }
-
-    @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();
@@ -336,6 +309,11 @@
                 hbData.gcCollectionCounts[i] = gcMXBean.getCollectionCount();
                 hbData.gcCollectionTimes[i] = gcMXBean.getCollectionTime();
             }
+            PerformanceCounters netPC = netManager.getPerformanceCounters();
+            hbData.netPayloadBytesRead = netPC.getPayloadBytesRead();
+            hbData.netPayloadBytesWritten = netPC.getPayloadBytesWritten();
+            hbData.netSignalingBytesRead = netPC.getSignalingBytesRead();
+            hbData.netSignalingBytesWritten = netPC.getSignalingBytesWritten();
             try {
                 cc.nodeHeartbeat(id, hbData);
             } catch (Exception e) {
@@ -366,4 +344,59 @@
             }
         }
     }
+
+    private final class NodeControllerIPCI implements IIPCI {
+        @Override
+        public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception) {
+            CCNCFunctions.Function fn = (CCNCFunctions.Function) payload;
+            switch (fn.getFunctionId()) {
+                case START_TASKS: {
+                    CCNCFunctions.StartTasksFunction stf = (CCNCFunctions.StartTasksFunction) fn;
+                    queue.schedule(new StartTasksWork(NodeControllerService.this, stf.getAppName(), stf.getJobId(), stf
+                            .getPlanBytes(), stf.getTaskDescriptors(), stf.getConnectorPolicies()));
+                    return;
+                }
+
+                case ABORT_TASKS: {
+                    CCNCFunctions.AbortTasksFunction atf = (CCNCFunctions.AbortTasksFunction) fn;
+                    queue.schedule(new AbortTasksWork(NodeControllerService.this, atf.getJobId(), atf.getTasks()));
+                    return;
+                }
+
+                case CLEANUP_JOBLET: {
+                    CCNCFunctions.CleanupJobletFunction cjf = (CCNCFunctions.CleanupJobletFunction) fn;
+                    queue.schedule(new CleanupJobletWork(NodeControllerService.this, cjf.getJobId(), cjf.getStatus()));
+                    return;
+                }
+
+                case CREATE_APPLICATION: {
+                    CCNCFunctions.CreateApplicationFunction caf = (CCNCFunctions.CreateApplicationFunction) fn;
+                    queue.schedule(new CreateApplicationWork(NodeControllerService.this, caf.getAppName(), caf
+                            .isDeployHar(), caf.getSerializedDistributedState()));
+                    return;
+                }
+
+                case DESTROY_APPLICATION: {
+                    CCNCFunctions.DestroyApplicationFunction daf = (CCNCFunctions.DestroyApplicationFunction) fn;
+                    queue.schedule(new DestroyApplicationWork(NodeControllerService.this, daf.getAppName()));
+                    return;
+                }
+
+                case REPORT_PARTITION_AVAILABILITY: {
+                    CCNCFunctions.ReportPartitionAvailabilityFunction rpaf = (CCNCFunctions.ReportPartitionAvailabilityFunction) fn;
+                    queue.schedule(new ReportPartitionAvailabilityWork(NodeControllerService.this, rpaf
+                            .getPartitionId(), rpaf.getNetworkAddress()));
+                    return;
+                }
+
+                case NODE_REGISTRATION_RESULT: {
+                    CCNCFunctions.NodeRegistrationResult nrrf = (CCNCFunctions.NodeRegistrationResult) fn;
+                    setNodeRegistrationResult(nrrf.getNodeParameters(), nrrf.getException());
+                    return;
+                }
+            }
+            throw new IllegalArgumentException("Unknown function: " + fn.getFunctionId());
+
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
deleted file mode 100644
index 6e38ef7..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
+++ /dev/null
@@ -1,254 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.nc.net;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.nio.ByteBuffer;
-import java.nio.channels.AsynchronousCloseException;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.ServerSocketChannel;
-import java.nio.channels.SocketChannel;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.NetworkAddress;
-import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.partitions.PartitionId;
-import edu.uci.ics.hyracks.control.nc.partitions.IPartitionRequestListener;
-
-public class ConnectionManager {
-    private static final Logger LOGGER = Logger.getLogger(ConnectionManager.class.getName());
-
-    static final int INITIAL_MESSAGE_SIZE = 20;
-
-    private final IHyracksRootContext ctx;
-
-    private IPartitionRequestListener partitionRequestListener;
-
-    private final ServerSocketChannel serverChannel;
-
-    private volatile boolean stopped;
-
-    private final ConnectionListenerThread connectionListener;
-
-    private final DataListenerThread dataListener;
-
-    private final NetworkAddress networkAddress;
-
-    public ConnectionManager(IHyracksRootContext ctx, InetAddress inetAddress) throws IOException {
-        this.ctx = ctx;
-        serverChannel = ServerSocketChannel.open();
-        ServerSocket serverSocket = serverChannel.socket();
-        serverSocket.bind(new InetSocketAddress(inetAddress, 0), 0);
-        serverSocket.setReuseAddress(true);
-        stopped = false;
-        connectionListener = new ConnectionListenerThread();
-        dataListener = new DataListenerThread();
-        networkAddress = new NetworkAddress(serverSocket.getInetAddress(), serverSocket.getLocalPort());
-
-    }
-
-    public void setPartitionRequestListener(IPartitionRequestListener partitionRequestListener) {
-        this.partitionRequestListener = partitionRequestListener;
-    }
-
-    public void start() {
-        connectionListener.start();
-        dataListener.start();
-    }
-
-    public void stop() {
-        try {
-            stopped = true;
-            serverChannel.close();
-        } catch (IOException e) {
-            e.printStackTrace();
-        }
-    }
-
-    public void connect(INetworkChannel channel) throws IOException {
-        dataListener.addOutgoingConnection(channel);
-    }
-
-    private final class ConnectionListenerThread extends Thread {
-        public ConnectionListenerThread() {
-            super("Hyracks NC Connection Listener");
-            setDaemon(true);
-            setPriority(MAX_PRIORITY);
-        }
-
-        @Override
-        public void run() {
-            while (!stopped) {
-                try {
-                    SocketChannel sc = serverChannel.accept();
-                    dataListener.addIncomingConnection(sc);
-                } catch (AsynchronousCloseException e) {
-                    // do nothing
-                    if (!stopped) {
-                        e.printStackTrace();
-                    }
-                } catch (IOException e) {
-                    e.printStackTrace();
-                }
-            }
-        }
-    }
-
-    private final class DataListenerThread extends Thread {
-        private Selector selector;
-
-        private final List<SocketChannel> pendingIncomingConnections;
-        private final Set<SocketChannel> pendingNegotiations;
-        private final List<INetworkChannel> pendingOutgoingConnections;
-
-        public DataListenerThread() {
-            super("Hyracks Data Listener Thread");
-            setDaemon(true);
-            try {
-                selector = Selector.open();
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
-            pendingIncomingConnections = new ArrayList<SocketChannel>();
-            pendingNegotiations = new HashSet<SocketChannel>();
-            pendingOutgoingConnections = new ArrayList<INetworkChannel>();
-        }
-
-        synchronized void addIncomingConnection(SocketChannel sc) throws IOException {
-            pendingIncomingConnections.add(sc);
-            selector.wakeup();
-        }
-
-        synchronized void addOutgoingConnection(INetworkChannel channel) throws IOException {
-            pendingOutgoingConnections.add(channel);
-            selector.wakeup();
-        }
-
-        @Override
-        public void run() {
-            while (!stopped) {
-                try {
-                    if (LOGGER.isLoggable(Level.FINE)) {
-                        LOGGER.fine("Starting Select");
-                    }
-                    int n = selector.select();
-                    synchronized (this) {
-                        if (!pendingIncomingConnections.isEmpty()) {
-                            for (SocketChannel sc : pendingIncomingConnections) {
-                                sc.configureBlocking(false);
-                                sc.socket().setReuseAddress(true);
-                                SelectionKey scKey = sc.register(selector, SelectionKey.OP_READ);
-                                ByteBuffer buffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
-                                scKey.attach(buffer);
-                                pendingNegotiations.add(sc);
-                            }
-                            pendingIncomingConnections.clear();
-                        }
-                        if (!pendingOutgoingConnections.isEmpty()) {
-                            for (INetworkChannel nc : pendingOutgoingConnections) {
-                                SocketChannel sc = SocketChannel.open();
-                                sc.configureBlocking(false);
-                                sc.socket().setReuseAddress(true);
-                                SelectionKey scKey = sc.register(selector, 0);
-                                scKey.attach(nc);
-                                nc.setSelectionKey(scKey);
-                                nc.notifyConnectionManagerRegistration();
-                            }
-                            pendingOutgoingConnections.clear();
-                        }
-                        if (LOGGER.isLoggable(Level.FINE)) {
-                            LOGGER.fine("Selector: " + n);
-                        }
-                        if (n > 0) {
-                            for (Iterator<SelectionKey> i = selector.selectedKeys().iterator(); i.hasNext();) {
-                                SelectionKey key = i.next();
-                                i.remove();
-                                SocketChannel sc = (SocketChannel) key.channel();
-                                if (pendingNegotiations.contains(sc)) {
-                                    if (key.isReadable()) {
-                                        ByteBuffer buffer = (ByteBuffer) key.attachment();
-                                        sc.read(buffer);
-                                        buffer.flip();
-                                        if (buffer.remaining() >= INITIAL_MESSAGE_SIZE) {
-                                            PartitionId pid = readInitialMessage(buffer);
-                                            pendingNegotiations.remove(sc);
-                                            key.interestOps(0);
-                                            NetworkOutputChannel channel = new NetworkOutputChannel(ctx, 5);
-                                            channel.setSelectionKey(key);
-                                            key.attach(channel);
-                                            try {
-                                                partitionRequestListener.registerPartitionRequest(pid, channel);
-                                            } catch (HyracksException e) {
-                                                key.cancel();
-                                                sc.close();
-                                                channel.abort();
-                                            }
-                                        } else {
-                                            buffer.compact();
-                                        }
-                                    }
-                                } else {
-                                    INetworkChannel channel = (INetworkChannel) key.attachment();
-                                    boolean close = false;
-                                    boolean error = false;
-                                    try {
-                                        close = channel.dispatchNetworkEvent();
-                                    } catch (IOException e) {
-                                        e.printStackTrace();
-                                        error = true;
-                                    }
-                                    if (close || error) {
-                                        key.cancel();
-                                        sc.close();
-                                        if (error) {
-                                            channel.abort();
-                                        }
-                                    }
-                                }
-                            }
-                        }
-                    }
-                } catch (Exception e) {
-                    e.printStackTrace();
-                }
-            }
-        }
-
-        private PartitionId readInitialMessage(ByteBuffer buffer) {
-            JobId jobId = new JobId(buffer.getLong());
-            ConnectorDescriptorId cdid = new ConnectorDescriptorId(buffer.getInt());
-            int senderIndex = buffer.getInt();
-            int receiverIndex = buffer.getInt();
-            return new PartitionId(jobId, cdid, senderIndex, receiverIndex);
-        }
-    }
-
-    public NetworkAddress getNetworkAddress() {
-        return networkAddress;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java
deleted file mode 100644
index 61cd91f..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java
+++ /dev/null
@@ -1,19 +0,0 @@
-package edu.uci.ics.hyracks.control.nc.net;
-
-import java.io.IOException;
-import java.net.SocketAddress;
-import java.nio.channels.SelectionKey;
-
-public interface INetworkChannel {
-    public boolean dispatchNetworkEvent() throws IOException;
-
-    public void setSelectionKey(SelectionKey key);
-
-    public SelectionKey getSelectionKey();
-
-    public SocketAddress getRemoteAddress();
-
-    public void abort();
-
-    public void notifyConnectionManagerRegistration() throws IOException;
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
index 23cf514..ae2cd37 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
@@ -14,11 +14,8 @@
  */
 package edu.uci.ics.hyracks.control.nc.net;
 
-import java.io.IOException;
 import java.net.SocketAddress;
 import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
 import java.util.ArrayDeque;
 import java.util.Queue;
 import java.util.logging.Level;
@@ -26,53 +23,43 @@
 
 import edu.uci.ics.hyracks.api.channels.IInputChannel;
 import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
 
-public class NetworkInputChannel implements IInputChannel, INetworkChannel {
+public class NetworkInputChannel implements IInputChannel {
     private static final Logger LOGGER = Logger.getLogger(NetworkInputChannel.class.getName());
 
-    private final ConnectionManager connectionManager;
+    private IHyracksRootContext ctx;
+
+    private final NetworkManager netManager;
 
     private final SocketAddress remoteAddress;
 
     private final PartitionId partitionId;
 
-    private final Queue<ByteBuffer> emptyQueue;
-
     private final Queue<ByteBuffer> fullQueue;
 
-    private SocketChannel socketChannel;
+    private final int nBuffers;
 
-    private SelectionKey key;
-
-    private ByteBuffer currentBuffer;
-
-    private boolean eos;
-
-    private boolean aborted;
+    private ChannelControlBlock ccb;
 
     private IInputChannelMonitor monitor;
 
     private Object attachment;
 
-    private ByteBuffer writeBuffer;
-
-    public NetworkInputChannel(IHyracksRootContext ctx, ConnectionManager connectionManager,
-            SocketAddress remoteAddress, PartitionId partitionId, int nBuffers) {
-        this.connectionManager = connectionManager;
+    public NetworkInputChannel(IHyracksRootContext ctx, NetworkManager netManager, SocketAddress remoteAddress,
+            PartitionId partitionId, int nBuffers) {
+        this.ctx = ctx;
+        this.netManager = netManager;
         this.remoteAddress = remoteAddress;
         this.partitionId = partitionId;
-        this.emptyQueue = new ArrayDeque<ByteBuffer>(nBuffers);
-        for (int i = 0; i < nBuffers; ++i) {
-            emptyQueue.add(ctx.allocateFrame());
-        }
         fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
-        aborted = false;
-        eos = false;
+        this.nBuffers = nBuffers;
     }
 
     @Override
@@ -96,29 +83,34 @@
     }
 
     @Override
-    public synchronized void recycleBuffer(ByteBuffer buffer) {
+    public void recycleBuffer(ByteBuffer buffer) {
         buffer.clear();
-        emptyQueue.add(buffer);
-        if (!eos && !aborted) {
-            int ops = key.interestOps();
-            if ((ops & SelectionKey.OP_READ) == 0) {
-                key.interestOps(ops | SelectionKey.OP_READ);
-                key.selector().wakeup();
-                if (currentBuffer == null) {
-                    currentBuffer = emptyQueue.poll();
-                }
-            }
-        }
+        ccb.getReadInterface().getEmptyBufferAcceptor().accept(buffer);
     }
 
     @Override
     public void open() throws HyracksDataException {
-        currentBuffer = emptyQueue.poll();
         try {
-            connectionManager.connect(this);
-        } catch (IOException e) {
+            ccb = netManager.connect(remoteAddress);
+        } catch (Exception e) {
             throw new HyracksDataException(e);
         }
+        ccb.getReadInterface().setFullBufferAcceptor(new ReadFullBufferAcceptor());
+        ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
+        for (int i = 0; i < nBuffers; ++i) {
+            ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
+        }
+        ByteBuffer writeBuffer = ByteBuffer.allocate(NetworkManager.INITIAL_MESSAGE_SIZE);
+        writeBuffer.putLong(partitionId.getJobId().getId());
+        writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
+        writeBuffer.putInt(partitionId.getSenderIndex());
+        writeBuffer.putInt(partitionId.getReceiverIndex());
+        writeBuffer.flip();
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Sending partition request: " + partitionId + " on channel: " + ccb);
+        }
+        ccb.getWriteInterface().getFullBufferAcceptor().accept(writeBuffer);
+        ccb.getWriteInterface().getFullBufferAcceptor().close();
     }
 
     @Override
@@ -126,110 +118,28 @@
 
     }
 
-    @Override
-    public synchronized boolean dispatchNetworkEvent() throws IOException {
-        if (aborted) {
-            eos = true;
-            monitor.notifyFailure(this);
-            return true;
+    private class ReadFullBufferAcceptor implements ICloseableBufferAcceptor {
+        @Override
+        public void accept(ByteBuffer buffer) {
+            fullQueue.add(buffer);
+            monitor.notifyDataAvailability(NetworkInputChannel.this, 1);
         }
-        if (key.isConnectable()) {
-            if (socketChannel.finishConnect()) {
-                key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT);
-                prepareForWrite();
-            }
-        } else if (key.isWritable()) {
-            socketChannel.write(writeBuffer);
-            if (writeBuffer.remaining() == 0) {
-                key.interestOps(SelectionKey.OP_READ);
-            }
-        } else if (key.isReadable()) {
-            if (LOGGER.isLoggable(Level.FINER)) {
-                LOGGER.finer("Before read: " + currentBuffer.position() + " " + currentBuffer.limit());
-            }
-            int bytesRead = socketChannel.read(currentBuffer);
-            if (bytesRead < 0) {
-                eos = true;
-                monitor.notifyEndOfStream(this);
-                return true;
-            }
-            if (LOGGER.isLoggable(Level.FINER)) {
-                LOGGER.finer("After read: " + currentBuffer.position() + " " + currentBuffer.limit());
-            }
-            currentBuffer.flip();
-            int dataLen = currentBuffer.remaining();
-            if (dataLen >= currentBuffer.capacity() || aborted()) {
-                if (LOGGER.isLoggable(Level.FINEST)) {
-                    LOGGER.finest("NetworkInputChannel: frame received: sender = " + partitionId.getSenderIndex());
-                }
-                if (currentBuffer.getInt(FrameHelper.getTupleCountOffset(currentBuffer.capacity())) == 0) {
-                    eos = true;
-                    monitor.notifyEndOfStream(this);
-                    return true;
-                }
-                fullQueue.add(currentBuffer);
-                currentBuffer = emptyQueue.poll();
-                if (currentBuffer == null && key.isValid()) {
-                    int ops = key.interestOps();
-                    key.interestOps(ops & ~SelectionKey.OP_READ);
-                }
-                monitor.notifyDataAvailability(this, 1);
-                return false;
-            }
-            currentBuffer.compact();
+
+        @Override
+        public void close() {
+            monitor.notifyEndOfStream(NetworkInputChannel.this);
         }
-        return false;
+
+        @Override
+        public void error(int ecode) {
+            monitor.notifyFailure(NetworkInputChannel.this);
+        }
     }
 
-    private void prepareForConnect() {
-        key.interestOps(SelectionKey.OP_CONNECT);
-    }
-
-    private void prepareForWrite() {
-        writeBuffer = ByteBuffer.allocate(ConnectionManager.INITIAL_MESSAGE_SIZE);
-        writeBuffer.putLong(partitionId.getJobId().getId());
-        writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
-        writeBuffer.putInt(partitionId.getSenderIndex());
-        writeBuffer.putInt(partitionId.getReceiverIndex());
-        writeBuffer.flip();
-
-        key.interestOps(SelectionKey.OP_WRITE);
-    }
-
-    @Override
-    public void setSelectionKey(SelectionKey key) {
-        this.key = key;
-        socketChannel = (SocketChannel) key.channel();
-    }
-
-    @Override
-    public SocketAddress getRemoteAddress() {
-        return remoteAddress;
-    }
-
-    @Override
-    public SelectionKey getSelectionKey() {
-        return key;
-    }
-
-    public PartitionId getPartitionId() {
-        return partitionId;
-    }
-
-    public void abort() {
-        aborted = true;
-    }
-
-    public boolean aborted() {
-        return aborted;
-    }
-
-    @Override
-    public void notifyConnectionManagerRegistration() throws IOException {
-        if (socketChannel.connect(remoteAddress)) {
-            prepareForWrite();
-        } else {
-            prepareForConnect();
+    private class WriteEmptyBufferAcceptor implements IBufferAcceptor {
+        @Override
+        public void accept(ByteBuffer buffer) {
+            // do nothing
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
new file mode 100644
index 0000000..1a0a820
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
@@ -0,0 +1,134 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.net;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.nc.partitions.IPartitionRequestListener;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.IChannelOpenListener;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.PerformanceCounters;
+
+public class NetworkManager {
+    private static final Logger LOGGER = Logger.getLogger(NetworkManager.class.getName());
+
+    static final int INITIAL_MESSAGE_SIZE = 20;
+
+    private final IHyracksRootContext ctx;
+
+    private final IPartitionRequestListener partitionRequestListener;
+
+    private final MuxDemux md;
+
+    private NetworkAddress networkAddress;
+
+    public NetworkManager(IHyracksRootContext ctx, InetAddress inetAddress,
+            IPartitionRequestListener partitionRequestListener, int nThreads) throws IOException {
+        this.ctx = ctx;
+        this.partitionRequestListener = partitionRequestListener;
+        md = new MuxDemux(new InetSocketAddress(inetAddress, 0), new ChannelOpenListener(), nThreads);
+    }
+
+    public void start() throws IOException {
+        md.start();
+        InetSocketAddress sockAddr = md.getLocalAddress();
+        networkAddress = new NetworkAddress(sockAddr.getAddress().getAddress(), sockAddr.getPort());
+    }
+
+    public NetworkAddress getNetworkAddress() {
+        return networkAddress;
+    }
+
+    public void stop() {
+
+    }
+
+    public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException {
+        MultiplexedConnection mConn = md.connect((InetSocketAddress) remoteAddress);
+        return mConn.openChannel();
+    }
+
+    private class ChannelOpenListener implements IChannelOpenListener {
+        @Override
+        public void channelOpened(ChannelControlBlock channel) {
+            channel.getReadInterface().setFullBufferAcceptor(new InitialBufferAcceptor(channel));
+            channel.getReadInterface().getEmptyBufferAcceptor().accept(ByteBuffer.allocate(INITIAL_MESSAGE_SIZE));
+        }
+    }
+
+    private class InitialBufferAcceptor implements ICloseableBufferAcceptor {
+        private final ChannelControlBlock ccb;
+
+        private NetworkOutputChannel noc;
+
+        public InitialBufferAcceptor(ChannelControlBlock ccb) {
+            this.ccb = ccb;
+        }
+
+        @Override
+        public void accept(ByteBuffer buffer) {
+            PartitionId pid = readInitialMessage(buffer);
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Received initial partition request: " + pid + " on channel: " + ccb);
+            }
+            noc = new NetworkOutputChannel(ctx, ccb, 5);
+            try {
+                partitionRequestListener.registerPartitionRequest(pid, noc);
+            } catch (HyracksException e) {
+                noc.abort();
+            }
+        }
+
+        @Override
+        public void close() {
+
+        }
+
+        @Override
+        public void error(int ecode) {
+            if (noc != null) {
+                noc.abort();
+            }
+        }
+    }
+
+    private static PartitionId readInitialMessage(ByteBuffer buffer) {
+        JobId jobId = new JobId(buffer.getLong());
+        ConnectorDescriptorId cdid = new ConnectorDescriptorId(buffer.getInt());
+        int senderIndex = buffer.getInt();
+        int receiverIndex = buffer.getInt();
+        return new PartitionId(jobId, cdid, senderIndex, receiverIndex);
+    }
+
+    public PerformanceCounters getPerformanceCounters() {
+        return md.getPerformanceCounters();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
index 31ce924..8d5f475 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
@@ -14,124 +14,44 @@
  */
 package edu.uci.ics.hyracks.control.nc.net;
 
-import java.io.IOException;
-import java.net.SocketAddress;
 import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
 import java.util.ArrayDeque;
 import java.util.Queue;
 
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
 
-public class NetworkOutputChannel implements INetworkChannel, IFrameWriter {
-    private final IHyracksRootContext ctx;
+public class NetworkOutputChannel implements IFrameWriter {
+    private final ChannelControlBlock ccb;
 
     private final Queue<ByteBuffer> emptyQueue;
 
-    private final Queue<ByteBuffer> fullQueue;
-
-    private SelectionKey key;
-
     private boolean aborted;
 
-    private boolean eos;
-
-    private boolean eosSent;
-
-    private boolean failed;
-
-    private ByteBuffer currentBuffer;
-
-    public NetworkOutputChannel(IHyracksRootContext ctx, int nBuffers) {
-        this.ctx = ctx;
+    public NetworkOutputChannel(IHyracksRootContext ctx, ChannelControlBlock ccb, int nBuffers) {
+        this.ccb = ccb;
         emptyQueue = new ArrayDeque<ByteBuffer>(nBuffers);
         for (int i = 0; i < nBuffers; ++i) {
-            emptyQueue.add(ctx.allocateFrame());
+            emptyQueue.add(ByteBuffer.allocateDirect(ctx.getFrameSize()));
         }
-        fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
-    }
-
-    @Override
-    public synchronized boolean dispatchNetworkEvent() throws IOException {
-        if (failed || aborted) {
-            eos = true;
-            return true;
-        } else if (key.isWritable()) {
-            while (true) {
-                if (currentBuffer == null) {
-                    if (eosSent) {
-                        return true;
-                    }
-                    currentBuffer = fullQueue.poll();
-                    if (currentBuffer == null) {
-                        if (eos) {
-                            currentBuffer = emptyQueue.poll();
-                            currentBuffer.clear();
-                            currentBuffer.putInt(FrameHelper.getTupleCountOffset(ctx.getFrameSize()), 0);
-                            eosSent = true;
-                        } else {
-                            key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE);
-                            return false;
-                        }
-                    }
-                }
-                int bytesWritten = ((SocketChannel) key.channel()).write(currentBuffer);
-                if (bytesWritten < 0) {
-                    eos = true;
-                    return true;
-                }
-                if (currentBuffer.remaining() == 0) {
-                    emptyQueue.add(currentBuffer);
-                    notifyAll();
-                    currentBuffer = null;
-                    if (eosSent) {
-                        return true;
-                    }
-                } else {
-                    return false;
-                }
-            }
-        }
-        return false;
-    }
-
-    @Override
-    public void setSelectionKey(SelectionKey key) {
-        this.key = key;
-    }
-
-    @Override
-    public SelectionKey getSelectionKey() {
-        return key;
-    }
-
-    @Override
-    public SocketAddress getRemoteAddress() {
-        return ((SocketChannel) key.channel()).socket().getRemoteSocketAddress();
-    }
-
-    @Override
-    public synchronized void abort() {
-        aborted = true;
+        ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
     }
 
     @Override
     public void open() throws HyracksDataException {
-        currentBuffer = null;
     }
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
         ByteBuffer destBuffer = null;
         synchronized (this) {
-            if (aborted) {
-                throw new HyracksDataException("Connection has been aborted");
-            }
             while (true) {
+                if (aborted) {
+                    throw new HyracksDataException("Connection has been aborted");
+                }
                 destBuffer = emptyQueue.poll();
                 if (destBuffer != null) {
                     break;
@@ -148,26 +68,34 @@
         destBuffer.clear();
         destBuffer.put(buffer);
         destBuffer.flip();
-        synchronized (this) {
-            fullQueue.add(destBuffer);
-        }
-        key.interestOps(SelectionKey.OP_WRITE);
-        key.selector().wakeup();
+        ccb.getWriteInterface().getFullBufferAcceptor().accept(destBuffer);
     }
 
     @Override
     public void fail() throws HyracksDataException {
-        failed = true;
+        ccb.getWriteInterface().getFullBufferAcceptor().error(1);
     }
 
     @Override
-    public synchronized void close() throws HyracksDataException {
-        eos = true;
-        key.interestOps(SelectionKey.OP_WRITE);
-        key.selector().wakeup();
+    public void close() throws HyracksDataException {
+        ccb.getWriteInterface().getFullBufferAcceptor().close();
     }
 
-    @Override
-    public void notifyConnectionManagerRegistration() throws IOException {
+    void abort() {
+        ccb.getWriteInterface().getFullBufferAcceptor().error(1);
+        synchronized (NetworkOutputChannel.this) {
+            aborted = true;
+            NetworkOutputChannel.this.notifyAll();
+        }
+    }
+
+    private class WriteEmptyBufferAcceptor implements IBufferAcceptor {
+        @Override
+        public void accept(ByteBuffer buffer) {
+            synchronized (NetworkOutputChannel.this) {
+                emptyQueue.add(buffer);
+                NetworkOutputChannel.this.notifyAll();
+            }
+        }
     }
 }
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
index 9403736..44e61f1 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
@@ -32,6 +32,8 @@
 
     private IFrameWriter delegate;
 
+    private boolean pendingConnection;
+
     private boolean failed;
 
     public PipelinedPartition(PartitionManager manager, PartitionId pid, TaskAttemptId taId) {
@@ -57,32 +59,44 @@
     }
 
     @Override
-    public synchronized void open() throws HyracksDataException {
+    public void open() throws HyracksDataException {
         manager.registerPartition(pid, taId, this, PartitionState.STARTED);
         failed = false;
-        while (delegate == null) {
-            try {
-                wait();
-            } catch (InterruptedException e) {
-                throw new HyracksDataException(e);
-            }
-        }
-        delegate.open();
+        pendingConnection = true;
     }
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        ensureConnected();
         delegate.nextFrame(buffer);
     }
 
+    private void ensureConnected() throws HyracksDataException {
+        if (pendingConnection) {
+            synchronized (this) {
+                while (delegate == null) {
+                    try {
+                        wait();
+                    } catch (InterruptedException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+            }
+            delegate.open();
+        }
+        pendingConnection = false;
+    }
+
     @Override
     public void fail() throws HyracksDataException {
+        ensureConnected();
         failed = true;
         delegate.fail();
     }
 
     @Override
     public void close() throws HyracksDataException {
+        ensureConnected();
         if (!failed) {
             manager.updatePartitionState(pid, taId, this, PartitionState.COMMITTED);
         }
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
index 5b60a08..eb982df 100644
--- 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
@@ -27,9 +27,9 @@
 import org.apache.http.impl.client.DefaultHttpClient;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
 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;
@@ -45,15 +45,12 @@
 
     private final byte[] serializedDistributedState;
 
-    private final FutureValue<Object> fv;
-
     public CreateApplicationWork(NodeControllerService ncs, String appName, boolean deployHar,
-            byte[] serializedDistributedState, FutureValue<Object> fv) {
+            byte[] serializedDistributedState) {
         this.ncs = ncs;
         this.appName = appName;
         this.deployHar = deployHar;
         this.serializedDistributedState = serializedDistributedState;
-        this.fv = fv;
     }
 
     @Override
@@ -85,9 +82,10 @@
             appCtx.initializeClassPath();
             appCtx.setDistributedState((Serializable) appCtx.deserialize(serializedDistributedState));
             appCtx.initialize();
-            fv.setValue(null);
+            ncs.getClusterController()
+                    .notifyApplicationStateChange(ncs.getId(), appName, ApplicationStatus.INITIALIZED);
         } catch (Exception e) {
-            fv.setException(e);
+            LOGGER.warning("Error creating application: " + e.getMessage());
         }
     }
 }
\ 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
index 8ac0b5c..cfe00f6 100644
--- 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
@@ -18,7 +18,7 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
-import edu.uci.ics.hyracks.control.common.work.FutureValue;
+import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
 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;
@@ -30,12 +30,9 @@
 
     private final String appName;
 
-    private FutureValue<Object> fv;
-
-    public DestroyApplicationWork(NodeControllerService ncs, String appName, FutureValue<Object> fv) {
+    public DestroyApplicationWork(NodeControllerService ncs, String appName) {
         this.ncs = ncs;
         this.appName = appName;
-        this.fv = fv;
     }
 
     @Override
@@ -46,9 +43,9 @@
             if (appCtx != null) {
                 appCtx.deinitialize();
             }
-            fv.setValue(null);
         } catch (Exception e) {
-            fv.setException(e);
+            LOGGER.warning("Error destroying application: " + e.getMessage());
         }
+        ncs.getClusterController().notifyApplicationStateChange(ncs.getId(), appName, ApplicationStatus.DEINITIALIZED);
     }
 }
\ 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
index bfa21c9..9734567 100644
--- 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
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.nc.work;
 
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.util.Map;
 import java.util.logging.Logger;
@@ -48,8 +49,8 @@
         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));
+                    ncs.getNetworkManager(), new InetSocketAddress(InetAddress.getByAddress(networkAddress
+                            .getIpAddress()), networkAddress.getPort()), pid, 5));
             ji.reportPartitionAvailability(channel);
         }
     }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
index 4ffc060..37154bc 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
@@ -28,21 +28,33 @@
     private static final long serialVersionUID = 1L;
     protected final ConnectorDescriptorId id;
 
+    protected String displayName;
+
     public AbstractConnectorDescriptor(JobSpecification spec) {
         this.id = spec.createConnectorDescriptor();
         spec.getConnectorMap().put(id, this);
+        displayName = getClass().getName() + "[" + id + "]";
     }
 
     public ConnectorDescriptorId getConnectorId() {
         return id;
     }
 
+    public String getDisplayName() {
+        return displayName;
+    }
+
+    public void setDisplayName(String displayName) {
+        this.displayName = displayName;
+    }
+
     @Override
     public JSONObject toJSON() throws JSONException {
         JSONObject jconn = new JSONObject();
 
         jconn.put("id", getConnectorId().getId());
         jconn.put("java-class", getClass().getName());
+        jconn.put("display-name", displayName);
 
         return jconn;
     }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
index 1bb13ca..1a60290 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
@@ -38,12 +38,15 @@
 
     protected final int outputArity;
 
+    protected String displayName;
+
     public AbstractOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity) {
         odId = spec.createOperatorDescriptorId();
         this.inputArity = inputArity;
         this.outputArity = outputArity;
         recordDescriptors = new RecordDescriptor[outputArity];
         spec.getOperatorMap().put(getOperatorId(), this);
+        displayName = getClass().getName() + "[" + odId + "]";
     }
 
     @Override
@@ -66,6 +69,14 @@
         return recordDescriptors;
     }
 
+    public String getDisplayName() {
+        return displayName;
+    }
+
+    public void setDisplayName(String displayName) {
+        this.displayName = displayName;
+    }
+
     @Override
     public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
             ICCApplicationContext appCtx) {
@@ -79,6 +90,7 @@
         jop.put("java-class", getClass().getName());
         jop.put("in-arity", getInputArity());
         jop.put("out-arity", getOutputArity());
+        jop.put("display-name", displayName);
         return jop;
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
index 2b37c7b..d5f4fd8 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
@@ -18,6 +18,8 @@
 import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.Collection;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.channels.IInputChannel;
 import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
@@ -31,6 +33,8 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 
 public class NonDeterministicPartitionCollector extends AbstractPartitionCollector {
+    private static final Logger LOGGER = Logger.getLogger(NonDeterministicPartitionCollector.class.getName());
+
     private final FrameReader reader;
 
     private final BitSet expectedPartitions;
@@ -173,6 +177,10 @@
                 int senderIndex = pid.getSenderIndex();
                 failSenders.set(senderIndex);
                 eosSenders.set(senderIndex);
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    LOGGER.fine("Failure: " + connectorId + " sender: " + senderIndex + " receiver: "
+                            + receiverIndex);
+                }
                 NonDeterministicPartitionCollector.this.notifyAll();
             }
         }
@@ -184,6 +192,10 @@
                 int senderIndex = pid.getSenderIndex();
                 availableFrameCounts[senderIndex] += nFrames;
                 frameAvailability.set(senderIndex);
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    LOGGER.fine("Data available: " + connectorId + " sender: " + senderIndex + " receiver: "
+                            + receiverIndex);
+                }
                 NonDeterministicPartitionCollector.this.notifyAll();
             }
         }
@@ -194,6 +206,10 @@
                 PartitionId pid = (PartitionId) channel.getAttachment();
                 int senderIndex = pid.getSenderIndex();
                 eosSenders.set(senderIndex);
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    LOGGER.fine("EOS: " + connectorId + " sender: " + senderIndex + " receiver: "
+                            + receiverIndex);
+                }
                 NonDeterministicPartitionCollector.this.notifyAll();
             }
         }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
index ca17779..41da49a 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
@@ -428,6 +428,10 @@
                                  */
 
                                 tupleBuilder.reset();
+                                
+                                for(int k = 0; k < storedKeys.length; k++){
+                                	tupleBuilder.addField(fta, tupleIndex, storedKeys[k]);
+                                }
 
                                 aggregator.init(tupleBuilder, fta, tupleIndex, aggregateState);
 
diff --git a/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml b/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
index 15d31b6..fe71db4 100644
--- a/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
+++ b/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
@@ -77,7 +77,7 @@
       <plugin>
       	<groupId>edu.uci.ics.hyracks</groupId>
       	<artifactId>hyracks-maven-plugin</artifactId>
-      	<version>0.0.1</version>
+      	<version>0.0.2</version>
         <configuration>
           <hyracksServerHome>${basedir}/../../../hyracks-server/target/hyracks-server-${project.version}-binary-assembly</hyracksServerHome>
           <hyracksCLIHome>${basedir}/../../../hyracks-cli/target/hyracks-cli-${project.version}-binary-assembly</hyracksCLIHome>
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index fac141c..4ed8361 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -116,7 +116,7 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(jobId.toString());
         }
-        cc.waitForCompletion(jobId);
+        hcc.waitForCompletion(jobId);
         dumpOutputFiles();
     }
 
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
similarity index 99%
rename from hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java
rename to hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
index 1c5c014..2813a62 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTests.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
@@ -64,7 +64,7 @@
 /**
  *
  */
-public class AggregationTests extends AbstractIntegrationTest {
+public class AggregationTest extends AbstractIntegrationTest {
 
     final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC2_ID,
             new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
diff --git a/hyracks-examples/text-example/textapp/pom.xml b/hyracks-examples/text-example/textapp/pom.xml
index 3b3b9fd..913a95a 100644
--- a/hyracks-examples/text-example/textapp/pom.xml
+++ b/hyracks-examples/text-example/textapp/pom.xml
@@ -77,7 +77,7 @@
       <plugin>
       	<groupId>edu.uci.ics.hyracks</groupId>
       	<artifactId>hyracks-maven-plugin</artifactId>
-      	<version>0.0.1</version>
+      	<version>0.0.2</version>
         <configuration>
           <hyracksServerHome>${basedir}/../../../hyracks-server/target/hyracks-server-${project.version}-binary-assembly</hyracksServerHome>
           <hyracksCLIHome>${basedir}/../../../hyracks-cli/target/hyracks-cli-${project.version}-binary-assembly</hyracksCLIHome>
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
index 81e9f84..eadf679 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
@@ -25,7 +25,7 @@
 
     private void initialize(Properties properties) throws Exception {
         String clusterController = (String) properties.get(ConfigurationConstants.clusterControllerHost);
-        connection = new HyracksConnection(clusterController, 1099);
+        connection = new HyracksConnection(clusterController, 1098);
         systemLibs = new HashSet<String>();
         for (String systemLib : ConfigurationConstants.systemLibs) {
             String systemLibPath = properties.getProperty(systemLib);
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCHandle.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCHandle.java
index a4adf23..8a3630f 100644
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCHandle.java
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCHandle.java
@@ -21,7 +21,7 @@
 public interface IIPCHandle {
     public InetSocketAddress getRemoteAddress();
 
-    public void send(Object request, IResponseCallback callback) throws IPCException;
+    public long send(long requestId, Object payload, Exception exception) throws IPCException;
 
     public void setAttachment(Object attachment);
 
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCI.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCI.java
index ba9f343..24ab943 100644
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCI.java
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IIPCI.java
@@ -15,5 +15,5 @@
 package edu.uci.ics.hyracks.ipc.api;
 
 public interface IIPCI {
-    public Object call(IIPCHandle caller, Object req) throws Exception;
+    public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception);
 }
\ No newline at end of file
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IPayloadSerializerDeserializer.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IPayloadSerializerDeserializer.java
new file mode 100644
index 0000000..62648ff
--- /dev/null
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/IPayloadSerializerDeserializer.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.ipc.api;
+
+import java.nio.ByteBuffer;
+
+public interface IPayloadSerializerDeserializer {
+    public Object deserializeObject(ByteBuffer buffer, int length) throws Exception;
+
+    public Exception deserializeException(ByteBuffer buffer, int length) throws Exception;
+
+    public byte[] serializeObject(Object object) throws Exception;
+
+    public byte[] serializeException(Exception object) throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/RPCInterface.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/RPCInterface.java
new file mode 100644
index 0000000..3340516
--- /dev/null
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/RPCInterface.java
@@ -0,0 +1,72 @@
+package edu.uci.ics.hyracks.ipc.api;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class RPCInterface implements IIPCI {
+    private final Map<Long, Request> reqMap;
+
+    public RPCInterface() {
+        reqMap = new HashMap<Long, RPCInterface.Request>();
+    }
+
+    public Object call(IIPCHandle handle, Object request) throws Exception {
+        Request req;
+        synchronized (this) {
+            req = new Request();
+            long mid = handle.send(-1, request, null);
+            reqMap.put(mid, req);
+        }
+        return req.getResponse();
+    }
+
+    @Override
+    public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception) {
+        Request req;
+        synchronized (this) {
+            req = reqMap.remove(rmid);
+        }
+        assert req != null;
+        if (exception != null) {
+            req.setException(exception);
+        } else {
+            req.setResult(payload);
+        }
+    }
+
+    private static class Request {
+        private boolean pending;
+
+        private Object result;
+
+        private Exception exception;
+
+        Request() {
+            pending = true;
+            result = null;
+            exception = null;
+        }
+
+        synchronized void setResult(Object result) {
+            this.pending = false;
+            this.result = result;
+            notifyAll();
+        }
+
+        synchronized void setException(Exception exception) {
+            this.pending = false;
+            this.exception = exception;
+            notifyAll();
+        }
+
+        synchronized Object getResponse() throws Exception {
+            while (pending) {
+                wait();
+            }
+            if (exception != null) {
+                throw exception;
+            }
+            return result;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/SyncRMI.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/SyncRMI.java
deleted file mode 100644
index 180b1dd..0000000
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/api/SyncRMI.java
+++ /dev/null
@@ -1,48 +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.ipc.api;
-
-public final class SyncRMI implements IResponseCallback {
-    private boolean pending;
-
-    private Object response;
-
-    private Exception exception;
-
-    public SyncRMI() {
-    }
-
-    @Override
-    public synchronized void callback(IIPCHandle handle, Object response, Exception exception) {
-        pending = false;
-        this.response = response;
-        this.exception = exception;
-        notifyAll();
-    }
-
-    public synchronized Object call(IIPCHandle handle, Object request) throws Exception {
-        pending = true;
-        response = null;
-        exception = null;
-        handle.send(request, this);
-        while (pending) {
-            wait();
-        }
-        if (exception != null) {
-            throw exception;
-        }
-        return response;
-    }
-}
\ No newline at end of file
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCConnectionManager.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCConnectionManager.java
index cbbe718..053ac6b 100644
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCConnectionManager.java
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCConnectionManager.java
@@ -43,13 +43,13 @@
 
     private final Map<InetSocketAddress, IPCHandle> ipcHandleMap;
 
-    private final List<IPCHandle>[] pendingConnections;
+    private final List<IPCHandle> pendingConnections;
 
-    private final List<Message>[] sendList;
+    private final List<IPCHandle> workingPendingConnections;
 
-    private int writerIndex;
+    private final List<Message> sendList;
 
-    private int readerIndex;
+    private final List<Message> workingSendList;
 
     private final InetSocketAddress address;
 
@@ -59,15 +59,16 @@
         this.system = system;
         this.networkThread = new NetworkThread();
         this.serverSocketChannel = ServerSocketChannel.open();
+        serverSocketChannel.socket().setReuseAddress(true);
         serverSocketChannel.configureBlocking(false);
         ServerSocket socket = serverSocketChannel.socket();
         socket.bind(socketAddress);
         address = new InetSocketAddress(socket.getInetAddress(), socket.getLocalPort());
         ipcHandleMap = new HashMap<InetSocketAddress, IPCHandle>();
-        pendingConnections = new ArrayList[] { new ArrayList<IPCHandle>(), new ArrayList<IPCHandle>() };
-        sendList = new ArrayList[] { new ArrayList<Message>(), new ArrayList<Message>() };
-        writerIndex = 0;
-        readerIndex = 1;
+        pendingConnections = new ArrayList<IPCHandle>();
+        workingPendingConnections = new ArrayList<IPCHandle>();
+        sendList = new ArrayList<Message>();
+        workingSendList = new ArrayList<Message>();
     }
 
     InetSocketAddress getAddress() {
@@ -90,7 +91,7 @@
             handle = ipcHandleMap.get(remoteAddress);
             if (handle == null) {
                 handle = new IPCHandle(system, remoteAddress);
-                pendingConnections[writerIndex].add(handle);
+                pendingConnections.add(handle);
                 networkThread.selector.wakeup();
             }
         }
@@ -103,14 +104,22 @@
     }
 
     synchronized void write(Message msg) {
-        sendList[writerIndex].add(msg);
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Enqueued message: " + msg);
+        }
+        sendList.add(msg);
         networkThread.selector.wakeup();
     }
 
-    private synchronized void swapReadersAndWriters() {
-        int temp = readerIndex;
-        readerIndex = writerIndex;
-        writerIndex = temp;
+    private synchronized void collectOutstandingWork() {
+        if (!pendingConnections.isEmpty()) {
+            workingPendingConnections.addAll(pendingConnections);
+            pendingConnections.clear();
+        }
+        if (!sendList.isEmpty()) {
+            workingSendList.addAll(sendList);
+            sendList.clear();
+        }
     }
 
     private Message createInitialReqMessage(IPCHandle handle) {
@@ -161,9 +170,9 @@
                         LOGGER.fine("Starting Select");
                     }
                     int n = selector.select();
-                    swapReadersAndWriters();
-                    if (!pendingConnections[readerIndex].isEmpty()) {
-                        for (IPCHandle handle : pendingConnections[readerIndex]) {
+                    collectOutstandingWork();
+                    if (!workingPendingConnections.isEmpty()) {
+                        for (IPCHandle handle : workingPendingConnections) {
                             SocketChannel channel = SocketChannel.open();
                             channel.configureBlocking(false);
                             SelectionKey cKey = null;
@@ -177,11 +186,14 @@
                             handle.setKey(cKey);
                             cKey.attach(handle);
                         }
-                        pendingConnections[readerIndex].clear();
+                        workingPendingConnections.clear();
                     }
-                    if (!sendList[readerIndex].isEmpty()) {
-                        for (Iterator<Message> i = sendList[readerIndex].iterator(); i.hasNext();) {
+                    if (!workingSendList.isEmpty()) {
+                        for (Iterator<Message> i = workingSendList.iterator(); i.hasNext();) {
                             Message msg = i.next();
+                            if (LOGGER.isLoggable(Level.FINE)) {
+                                LOGGER.fine("Processing send of message: " + msg);
+                            }
                             IPCHandle handle = msg.getIPCHandle();
                             if (handle.getState() == HandleState.CLOSED) {
                                 i.remove();
@@ -196,7 +208,7 @@
                                         SelectionKey key = handle.getKey();
                                         key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
                                     } else {
-                                        if (buffer.position() == 0) {
+                                        if (!buffer.hasRemaining()) {
                                             handle.resizeOutBuffer();
                                             continue;
                                         }
@@ -239,7 +251,10 @@
                                 } else if (!writeBuffer.hasRemaining()) {
                                     key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE);
                                 }
-                                handle.clearFull();
+                                if (handle.full()) {
+                                    handle.clearFull();
+                                    selector.wakeup();
+                                }
                             } else if (key.isAcceptable()) {
                                 assert sc == serverSocketChannel;
                                 SocketChannel channel = serverSocketChannel.accept();
@@ -261,7 +276,7 @@
                             }
                         }
                     }
-                } catch (IOException e) {
+                } catch (Exception e) {
                     e.printStackTrace();
                 }
             }
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCHandle.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCHandle.java
index 481a0b0..3d3bc7a 100644
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCHandle.java
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCHandle.java
@@ -17,11 +17,8 @@
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.SelectionKey;
-import java.util.HashMap;
-import java.util.Map;
 
 import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.IResponseCallback;
 import edu.uci.ics.hyracks.ipc.exceptions.IPCException;
 
 final class IPCHandle implements IIPCHandle {
@@ -31,8 +28,6 @@
 
     private InetSocketAddress remoteAddress;
 
-    private final Map<Long, IResponseCallback> pendingRequestMap;
-
     private HandleState state;
 
     private SelectionKey key;
@@ -48,7 +43,6 @@
     IPCHandle(IPCSystem system, InetSocketAddress remoteAddress) {
         this.system = system;
         this.remoteAddress = remoteAddress;
-        pendingRequestMap = new HashMap<Long, IResponseCallback>();
         inBuffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
         outBuffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE);
         outBuffer.flip();
@@ -60,24 +54,32 @@
         return remoteAddress;
     }
 
+    IPCSystem getIPCSystem() {
+        return system;
+    }
+
     void setRemoteAddress(InetSocketAddress remoteAddress) {
         this.remoteAddress = remoteAddress;
     }
 
     @Override
-    public synchronized void send(Object req, IResponseCallback callback) throws IPCException {
+    public synchronized long send(long requestId, Object req, Exception exception) throws IPCException {
         if (state != HandleState.CONNECTED) {
             throw new IPCException("Handle is not in Connected state");
         }
         Message msg = new Message(this);
         long mid = system.createMessageId();
         msg.setMessageId(mid);
-        msg.setRequestMessageId(-1);
-        msg.setPayload(req);
-        if (callback != null) {
-            pendingRequestMap.put(mid, callback);
+        msg.setRequestMessageId(requestId);
+        if (exception != null) {
+            msg.setFlag(Message.ERROR);
+            msg.setPayload(exception);
+        } else {
+            msg.setFlag(Message.NORMAL);
+            msg.setPayload(req);
         }
         system.getConnectionManager().write(msg);
+        return mid;
     }
 
     @Override
@@ -110,7 +112,7 @@
         this.state = state;
         notifyAll();
     }
-    
+
     synchronized void waitTillConnected() throws InterruptedException {
         while (!isConnected()) {
             wait();
@@ -127,9 +129,6 @@
 
     synchronized void close() {
         setState(HandleState.CLOSED);
-        for (IResponseCallback cb : pendingRequestMap.values()) {
-            cb.callback(this, null, new IPCException("IPC Handle Closed"));
-        }
     }
 
     synchronized void processIncomingMessages() {
@@ -157,19 +156,7 @@
                 }
                 continue;
             }
-            long requestMessageId = message.getRequestMessageId();
-            if (requestMessageId < 0) {
-                system.deliverIncomingMessage(message);
-            } else {
-                Long rid = Long.valueOf(requestMessageId);
-                IResponseCallback cb = pendingRequestMap.remove(rid);
-                if (cb != null) {
-                    byte flag = message.getFlag();
-                    Object payload = flag == Message.ERROR ? null : message.getPayload();
-                    Exception exception = (Exception) (flag == Message.ERROR ? message.getPayload() : null);
-                    cb.callback(this, payload, exception);
-                }
-            }
+            system.deliverIncomingMessage(message);
         }
         inBuffer.compact();
     }
@@ -178,14 +165,12 @@
         inBuffer.flip();
         ByteBuffer readBuffer = ByteBuffer.allocate(inBuffer.capacity() * 2);
         readBuffer.put(inBuffer);
-        readBuffer.compact();
         inBuffer = readBuffer;
     }
 
     void resizeOutBuffer() {
         ByteBuffer writeBuffer = ByteBuffer.allocate(outBuffer.capacity() * 2);
         writeBuffer.put(outBuffer);
-        writeBuffer.compact();
         writeBuffer.flip();
         outBuffer = writeBuffer;
     }
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCSystem.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCSystem.java
index 9eef8ff..6c9c82c 100644
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCSystem.java
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCSystem.java
@@ -16,11 +16,11 @@
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.concurrent.Executor;
 import java.util.concurrent.atomic.AtomicLong;
 
 import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
 import edu.uci.ics.hyracks.ipc.api.IIPCI;
+import edu.uci.ics.hyracks.ipc.api.IPayloadSerializerDeserializer;
 import edu.uci.ics.hyracks.ipc.exceptions.IPCException;
 
 public class IPCSystem {
@@ -28,18 +28,15 @@
 
     private final IIPCI ipci;
 
-    private final Executor executor;
+    private final IPayloadSerializerDeserializer serde;
 
     private final AtomicLong midFactory;
 
-    public IPCSystem(InetSocketAddress socketAddress) throws IOException {
-        this(socketAddress, null, null);
-    }
-
-    public IPCSystem(InetSocketAddress socketAddress, IIPCI ipci, Executor executor) throws IOException {
+    public IPCSystem(InetSocketAddress socketAddress, IIPCI ipci, IPayloadSerializerDeserializer serde)
+            throws IOException {
         cMgr = new IPCConnectionManager(this, socketAddress);
         this.ipci = ipci;
-        this.executor = executor;
+        this.serde = serde;
         midFactory = new AtomicLong();
     }
 
@@ -61,30 +58,25 @@
         }
     }
 
+    IPayloadSerializerDeserializer getSerializerDeserializer() {
+        return serde;
+    }
+
     long createMessageId() {
         return midFactory.incrementAndGet();
     }
 
     void deliverIncomingMessage(final Message message) {
-        assert message.getFlag() == Message.NORMAL;
-        executor.execute(new Runnable() {
-            @Override
-            public void run() {
-                IPCHandle handle = message.getIPCHandle();
-                Message response = new Message(handle);
-                response.setMessageId(createMessageId());
-                response.setRequestMessageId(message.getMessageId());
-                response.setFlag(Message.NORMAL);
-                try {
-                    Object result = ipci.call(handle, message.getPayload());
-                    response.setPayload(result);
-                } catch (Exception e) {
-                    response.setFlag(Message.ERROR);
-                    response.setPayload(e);
-                }
-                cMgr.write(response);
-            }
-        });
+        long mid = message.getMessageId();
+        long rmid = message.getRequestMessageId();
+        Object payload = null;
+        Exception exception = null;
+        if (message.getFlag() == Message.ERROR) {
+            exception = (Exception) message.getPayload();
+        } else {
+            payload = message.getPayload();
+        }
+        ipci.deliverIncomingMessage(message.getIPCHandle(), mid, rmid, payload, exception);
     }
 
     IPCConnectionManager getConnectionManager() {
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/JavaSerializationBasedPayloadSerializerDeserializer.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/JavaSerializationBasedPayloadSerializerDeserializer.java
new file mode 100644
index 0000000..fdf8e92
--- /dev/null
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/JavaSerializationBasedPayloadSerializerDeserializer.java
@@ -0,0 +1,53 @@
+package edu.uci.ics.hyracks.ipc.impl;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.ipc.api.IPayloadSerializerDeserializer;
+
+public class JavaSerializationBasedPayloadSerializerDeserializer implements IPayloadSerializerDeserializer {
+    @Override
+    public Object deserializeObject(ByteBuffer buffer, int length) throws Exception {
+        return deserialize(buffer, length);
+    }
+
+    @Override
+    public Exception deserializeException(ByteBuffer buffer, int length) throws Exception {
+        return (Exception) deserialize(buffer, length);
+    }
+
+    @Override
+    public byte[] serializeObject(Object object) throws Exception {
+        return serialize(object);
+    }
+
+    @Override
+    public byte[] serializeException(Exception exception) throws Exception {
+        return serialize(exception);
+    }
+
+    public static void serialize(OutputStream out, Object object) throws Exception {
+        ObjectOutputStream oos = new ObjectOutputStream(out);
+        oos.writeObject(object);
+        oos.flush();
+    }
+
+    private Object deserialize(ByteBuffer buffer, int length) throws Exception {
+        ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(buffer.array(), buffer.position(),
+                length));
+        Object object = ois.readObject();
+        ois.close();
+        return object;
+    }
+
+    private byte[] serialize(Object object) throws Exception {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        serialize(baos, object);
+        baos.close();
+        return baos.toByteArray();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/Message.java b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/Message.java
index ab3428e..6bb3156 100644
--- a/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/Message.java
+++ b/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/Message.java
@@ -14,13 +14,10 @@
  */
 package edu.uci.ics.hyracks.ipc.impl;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.ipc.api.IPayloadSerializerDeserializer;
+
 class Message {
     private static final int MSG_SIZE_SIZE = 4;
 
@@ -92,29 +89,26 @@
         return buffer.remaining() >= msgSize + MSG_SIZE_SIZE;
     }
 
-    void read(ByteBuffer buffer) throws IOException, ClassNotFoundException {
+    void read(ByteBuffer buffer) throws Exception {
         assert hasMessage(buffer);
         int msgSize = buffer.getInt();
         messageId = buffer.getLong();
         requestMessageId = buffer.getLong();
         flag = buffer.get();
         int finalPosition = buffer.position() + msgSize - HEADER_SIZE;
+        int length = msgSize - HEADER_SIZE;
         try {
-            ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(buffer.array(), buffer.position(),
-                    msgSize - HEADER_SIZE));
-            payload = ois.readObject();
-            ois.close();
+            IPayloadSerializerDeserializer serde = ipcHandle.getIPCSystem().getSerializerDeserializer();
+            payload = flag == ERROR ? serde.deserializeException(buffer, length) : serde.deserializeObject(buffer,
+                    length);
         } finally {
             buffer.position(finalPosition);
         }
     }
 
-    boolean write(ByteBuffer buffer) throws IOException {
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        ObjectOutputStream oos = new ObjectOutputStream(baos);
-        oos.writeObject(payload);
-        oos.close();
-        byte[] bytes = baos.toByteArray();
+    boolean write(ByteBuffer buffer) throws Exception {
+        IPayloadSerializerDeserializer serde = ipcHandle.getIPCSystem().getSerializerDeserializer();
+        byte[] bytes = flag == ERROR ? serde.serializeException((Exception) payload) : serde.serializeObject(payload);
         if (buffer.remaining() >= MSG_SIZE_SIZE + HEADER_SIZE + bytes.length) {
             buffer.putInt(HEADER_SIZE + bytes.length);
             buffer.putLong(messageId);
@@ -125,4 +119,9 @@
         }
         return false;
     }
+
+    @Override
+    public String toString() {
+        return "MSG[" + messageId + ":" + requestMessageId + ":" + flag + ":" + payload + "]";
+    }
 }
\ No newline at end of file
diff --git a/hyracks-ipc/src/test/java/edu/uci/ics/hyracks/ipc/tests/IPCTest.java b/hyracks-ipc/src/test/java/edu/uci/ics/hyracks/ipc/tests/IPCTest.java
new file mode 100644
index 0000000..5b2f660
--- /dev/null
+++ b/hyracks-ipc/src/test/java/edu/uci/ics/hyracks/ipc/tests/IPCTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.ipc.tests;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.IIPCI;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
+import edu.uci.ics.hyracks.ipc.exceptions.IPCException;
+import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
+import edu.uci.ics.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
+
+public class IPCTest {
+    @Test
+    public void test() throws Exception {
+        IPCSystem server = createServerIPCSystem();
+        server.start();
+        InetSocketAddress serverAddr = server.getSocketAddress();
+
+        RPCInterface rpci = new RPCInterface();
+        IPCSystem client = createClientIPCSystem(rpci);
+        client.start();
+
+        IIPCHandle handle = client.getHandle(serverAddr);
+
+        for (int i = 0; i < 100; ++i) {
+            Assert.assertEquals(rpci.call(handle, Integer.valueOf(i)), Integer.valueOf(2 * i));
+        }
+
+        try {
+            rpci.call(handle, "Foo");
+            Assert.assertTrue(false);
+        } catch (Exception e) {
+            Assert.assertTrue(true);
+        }
+    }
+
+    private IPCSystem createServerIPCSystem() throws IOException {
+        final Executor executor = Executors.newCachedThreadPool();
+        IIPCI ipci = new IIPCI() {
+            @Override
+            public void deliverIncomingMessage(final IIPCHandle handle, final long mid, long rmid,
+                    final Object payload, Exception exception) {
+                executor.execute(new Runnable() {
+                    @Override
+                    public void run() {
+                        Object result = null;
+                        Exception exception = null;
+                        try {
+                            Integer i = (Integer) payload;
+                            result = i.intValue() * 2;
+                        } catch (Exception e) {
+                            exception = e;
+                        }
+                        try {
+                            handle.send(mid, result, exception);
+                        } catch (IPCException e) {
+                            e.printStackTrace();
+                        }
+                    }
+                });
+            }
+        };
+        return new IPCSystem(new InetSocketAddress("127.0.0.1", 0), ipci,
+                new JavaSerializationBasedPayloadSerializerDeserializer());
+    }
+
+    private IPCSystem createClientIPCSystem(RPCInterface rpci) throws IOException {
+        return new IPCSystem(new InetSocketAddress("127.0.0.1", 0), rpci,
+                new JavaSerializationBasedPayloadSerializerDeserializer());
+    }
+}
\ No newline at end of file
diff --git a/hyracks-ipc/src/test/java/edu/uci/ics/ipc/tests/IPCTest.java b/hyracks-ipc/src/test/java/edu/uci/ics/ipc/tests/IPCTest.java
deleted file mode 100644
index 1fc1f6f..0000000
--- a/hyracks-ipc/src/test/java/edu/uci/ics/ipc/tests/IPCTest.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.ipc.tests;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Executors;
-
-import junit.framework.Assert;
-
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
-import edu.uci.ics.hyracks.ipc.api.IIPCI;
-import edu.uci.ics.hyracks.ipc.api.SyncRMI;
-import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
-
-public class IPCTest {
-    @Test
-    public void test() throws Exception {
-        IPCSystem server = createServerIPCSystem();
-        server.start();
-        InetSocketAddress serverAddr = server.getSocketAddress();
-
-        IPCSystem client = createClientIPCSystem();
-        client.start();
-
-        IIPCHandle handle = client.getHandle(serverAddr);
-
-        SyncRMI rmi = new SyncRMI();
-        for (int i = 0; i < 100; ++i) {
-            Assert.assertEquals(rmi.call(handle, Integer.valueOf(i)), Integer.valueOf(2 * i));
-        }
-
-        IIPCHandle rHandle = server.getHandle(client.getSocketAddress());
-
-        try {
-            rmi.call(rHandle, "Foo");
-            Assert.assertTrue(false);
-        } catch (Exception e) {
-            Assert.assertTrue(true);
-        }
-    }
-
-    private IPCSystem createServerIPCSystem() throws IOException {
-        Executor executor = Executors.newCachedThreadPool();
-        IIPCI ipci = new IIPCI() {
-            @Override
-            public Object call(IIPCHandle caller, Object req) throws Exception {
-                Integer i = (Integer) req;
-                return i.intValue() * 2;
-            }
-        };
-        return new IPCSystem(new InetSocketAddress("127.0.0.1", 0), ipci, executor);
-    }
-
-    private IPCSystem createClientIPCSystem() throws IOException {
-        Executor executor = Executors.newCachedThreadPool();
-        IIPCI ipci = new IIPCI() {
-            @Override
-            public Object call(IIPCHandle caller, Object req) throws Exception {
-                throw new IllegalStateException();
-            }
-        };
-        return new IPCSystem(new InetSocketAddress("127.0.0.1", 0), ipci, executor);
-    }
-}
\ No newline at end of file
diff --git a/hyracks-net/pom.xml b/hyracks-net/pom.xml
new file mode 100644
index 0000000..12004f7
--- /dev/null
+++ b/hyracks-net/pom.xml
@@ -0,0 +1,31 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>hyracks-net</artifactId>
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <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>
+    </plugins>
+  </build>
+  <dependencies>
+  <dependency>
+  	<groupId>junit</groupId>
+  	<artifactId>junit</artifactId>
+  	<version>4.8.1</version>
+  	<scope>test</scope>
+  </dependency>
+  </dependencies>
+</project>
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/buffers/IBufferAcceptor.java
similarity index 80%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/buffers/IBufferAcceptor.java
index c728b0b..2f27bf0 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/buffers/IBufferAcceptor.java
@@ -12,10 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.buffers;
 
-public interface Accumulator<T, R> {
-    public void accumulate(T o);
+import java.nio.ByteBuffer;
 
-    public R getResult();
+public interface IBufferAcceptor {
+    public void accept(ByteBuffer buffer);
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/buffers/ICloseableBufferAcceptor.java
similarity index 78%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/buffers/ICloseableBufferAcceptor.java
index c728b0b..c395ac9 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/buffers/ICloseableBufferAcceptor.java
@@ -12,10 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.buffers;
 
-public interface Accumulator<T, R> {
-    public void accumulate(T o);
+public interface ICloseableBufferAcceptor extends IBufferAcceptor {
+    public void close();
 
-    public R getResult();
+    public void error(int ecode);
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/exceptions/NetException.java
similarity index 60%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/exceptions/NetException.java
index ec15186..ecd0373 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/exceptions/NetException.java
@@ -12,12 +12,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.exceptions;
 
-import edu.uci.ics.hyracks.control.common.base.INodeController;
+public class NetException extends Exception {
+    private static final long serialVersionUID = 1L;
 
-public interface RemoteOp<T> {
-    public String getNodeId();
+    public NetException() {
+    }
 
-    public T execute(INodeController node) throws Exception;
+    public NetException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public NetException(String message) {
+        super(message);
+    }
+
+    public NetException(Throwable cause) {
+        super(cause);
+    }
 }
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelControlBlock.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelControlBlock.java
new file mode 100644
index 0000000..7f6853b
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelControlBlock.java
@@ -0,0 +1,330 @@
+/*
+ * 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.net.protocols.muxdemux;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+
+public class ChannelControlBlock {
+    private static final Logger LOGGER = Logger.getLogger(ChannelControlBlock.class.getName());
+
+    private final ChannelSet cSet;
+
+    private final int channelId;
+
+    private final ReadInterface ri;
+
+    private final WriteInterface wi;
+
+    private final AtomicBoolean localClose;
+
+    private final AtomicBoolean localCloseAck;
+
+    private final AtomicBoolean remoteClose;
+
+    ChannelControlBlock(ChannelSet cSet, int channelId) {
+        this.cSet = cSet;
+        this.channelId = channelId;
+        this.ri = new ReadInterface();
+        this.wi = new WriteInterface();
+        localClose = new AtomicBoolean();
+        localCloseAck = new AtomicBoolean();
+        remoteClose = new AtomicBoolean();
+    }
+
+    int getChannelId() {
+        return channelId;
+    }
+
+    public IChannelReadInterface getReadInterface() {
+        return ri;
+    }
+
+    public IChannelWriteInterface getWriteInterface() {
+        return wi;
+    }
+
+    private final class ReadInterface implements IChannelReadInterface {
+        private final Queue<ByteBuffer> riEmptyQueue;
+
+        private final IBufferAcceptor eba = new IBufferAcceptor() {
+            @Override
+            public void accept(ByteBuffer buffer) {
+                if (remoteClose.get()) {
+                    return;
+                }
+                int delta;
+                synchronized (ChannelControlBlock.this) {
+                    riEmptyQueue.add(buffer);
+                    delta = buffer.remaining();
+                }
+                credits.addAndGet(delta);
+                if (delta != 0) {
+                    cSet.markPendingCredits(channelId);
+                }
+            }
+        };
+
+        private ICloseableBufferAcceptor fba;
+
+        private final AtomicInteger credits;
+
+        private ByteBuffer currentReadBuffer;
+
+        ReadInterface() {
+            riEmptyQueue = new LinkedList<ByteBuffer>();
+            credits = new AtomicInteger();
+        }
+
+        @Override
+        public void setFullBufferAcceptor(ICloseableBufferAcceptor fullBufferAcceptor) {
+            fba = fullBufferAcceptor;
+        }
+
+        @Override
+        public IBufferAcceptor getEmptyBufferAcceptor() {
+            return eba;
+        }
+
+        int read(SocketChannel sc, int size) throws IOException, NetException {
+            while (true) {
+                if (size <= 0) {
+                    return size;
+                }
+                if (ri.currentReadBuffer == null) {
+                    ri.currentReadBuffer = ri.riEmptyQueue.poll();
+                    assert ri.currentReadBuffer != null;
+                }
+                int rSize = Math.min(size, ri.currentReadBuffer.remaining());
+                if (rSize > 0) {
+                    ri.currentReadBuffer.limit(ri.currentReadBuffer.position() + rSize);
+                    int len;
+                    try {
+                        len = sc.read(ri.currentReadBuffer);
+                        if (len < 0) {
+                            throw new NetException("Socket Closed");
+                        }
+                    } finally {
+                        ri.currentReadBuffer.limit(ri.currentReadBuffer.capacity());
+                    }
+                    size -= len;
+                    if (len < rSize) {
+                        return size;
+                    }
+                } else {
+                    return size;
+                }
+                if (ri.currentReadBuffer.remaining() <= 0) {
+                    flush();
+                }
+            }
+        }
+
+        void flush() {
+            if (currentReadBuffer != null) {
+                currentReadBuffer.flip();
+                fba.accept(ri.currentReadBuffer);
+                currentReadBuffer = null;
+            }
+        }
+    }
+
+    private final class WriteInterface implements IChannelWriteInterface {
+        private final Queue<ByteBuffer> wiFullQueue;
+
+        private int channelWriteEventCount;
+
+        private final ICloseableBufferAcceptor fba = new ICloseableBufferAcceptor() {
+            @Override
+            public void accept(ByteBuffer buffer) {
+                synchronized (ChannelControlBlock.this) {
+                    wiFullQueue.add(buffer);
+                    incrementLocalWriteEventCount();
+                }
+            }
+
+            @Override
+            public void close() {
+                synchronized (ChannelControlBlock.this) {
+                    if (eos) {
+                        if (LOGGER.isLoggable(Level.WARNING)) {
+                            LOGGER.warning("Received duplicate close() on channel: " + channelId);
+                        }
+                        return;
+                    }
+                    eos = true;
+                    incrementLocalWriteEventCount();
+                }
+            }
+
+            @Override
+            public void error(int ecode) {
+                synchronized (ChannelControlBlock.this) {
+                    WriteInterface.this.ecode = ecode;
+                    incrementLocalWriteEventCount();
+                }
+            }
+        };
+
+        private IBufferAcceptor eba;
+
+        private final AtomicInteger credits;
+
+        private boolean eos;
+
+        private boolean eosSent;
+
+        private int ecode;
+
+        private boolean ecodeSent;
+
+        private ByteBuffer currentWriteBuffer;
+
+        WriteInterface() {
+            wiFullQueue = new LinkedList<ByteBuffer>();
+            credits = new AtomicInteger();
+            eos = false;
+            eosSent = false;
+            ecode = -1;
+            ecodeSent = false;
+        }
+
+        @Override
+        public void setEmptyBufferAcceptor(IBufferAcceptor emptyBufferAcceptor) {
+            eba = emptyBufferAcceptor;
+        }
+
+        @Override
+        public ICloseableBufferAcceptor getFullBufferAcceptor() {
+            return fba;
+        }
+
+        void write(MultiplexedConnection.WriterState writerState) throws NetException {
+            if (currentWriteBuffer == null) {
+                currentWriteBuffer = wiFullQueue.poll();
+            }
+            if (currentWriteBuffer != null) {
+                int size = Math.min(currentWriteBuffer.remaining(), credits.get());
+                if (size > 0) {
+                    credits.addAndGet(-size);
+                    writerState.command.setChannelId(channelId);
+                    writerState.command.setCommandType(MuxDemuxCommand.CommandType.DATA);
+                    writerState.command.setData(size);
+                    writerState.reset(currentWriteBuffer, size, ChannelControlBlock.this);
+                }
+            } else if (ecode >= 0 && !ecodeSent) {
+                decrementLocalWriteEventCount();
+                writerState.command.setChannelId(channelId);
+                writerState.command.setCommandType(MuxDemuxCommand.CommandType.ERROR);
+                writerState.command.setData(ecode);
+                writerState.reset(null, 0, null);
+                ecodeSent = true;
+                localClose.set(true);
+            } else if (wi.eos && !wi.eosSent) {
+                decrementLocalWriteEventCount();
+                writerState.command.setChannelId(channelId);
+                writerState.command.setCommandType(MuxDemuxCommand.CommandType.CLOSE_CHANNEL);
+                writerState.command.setData(0);
+                writerState.reset(null, 0, null);
+                eosSent = true;
+                localClose.set(true);
+            }
+        }
+
+        void writeComplete() {
+            if (currentWriteBuffer.remaining() <= 0) {
+                currentWriteBuffer.clear();
+                eba.accept(currentWriteBuffer);
+                decrementLocalWriteEventCount();
+                currentWriteBuffer = null;
+            }
+        }
+
+        void incrementLocalWriteEventCount() {
+            ++channelWriteEventCount;
+            if (channelWriteEventCount == 1) {
+                cSet.markPendingWrite(channelId);
+            }
+        }
+
+        void decrementLocalWriteEventCount() {
+            --channelWriteEventCount;
+            if (channelWriteEventCount == 0) {
+                cSet.unmarkPendingWrite(channelId);
+            }
+        }
+    }
+
+    synchronized void write(MultiplexedConnection.WriterState writerState) throws NetException {
+        wi.write(writerState);
+    }
+
+    synchronized void writeComplete() {
+        wi.writeComplete();
+    }
+
+    synchronized int read(SocketChannel sc, int size) throws IOException, NetException {
+        return ri.read(sc, size);
+    }
+
+    void addReadCredits(int delta) {
+        ri.credits.addAndGet(delta);
+    }
+
+    int getAndResetReadCredits() {
+        return ri.credits.getAndSet(0);
+    }
+
+    void addWriteCredits(int delta) {
+        wi.credits.addAndGet(delta);
+    }
+
+    synchronized void reportRemoteEOS() {
+        ri.flush();
+        ri.fba.close();
+        remoteClose.set(true);
+    }
+
+    synchronized void reportLocalEOSAck() {
+        localCloseAck.set(true);
+    }
+
+    synchronized void reportRemoteError(int ecode) {
+        ri.flush();
+        ri.fba.error(ecode);
+        remoteClose.set(true);
+    }
+
+    boolean completelyClosed() {
+        return localCloseAck.get() && remoteClose.get();
+    }
+
+    @Override
+    public String toString() {
+        return "Channel:" + channelId + "[localClose: " + localClose + " localCloseAck: " + localCloseAck
+                + " remoteClose: " + remoteClose + " readCredits: " + ri.credits + " writeCredits: " + wi.credits + "]";
+    }
+}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java
new file mode 100644
index 0000000..9411d42
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java
@@ -0,0 +1,206 @@
+/*
+ * 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.net.protocols.muxdemux;
+
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+
+public class ChannelSet {
+    private static final Logger LOGGER = Logger.getLogger(ChannelSet.class.getName());
+
+    private static final int MAX_OPEN_CHANNELS = 1024;
+
+    private static final int INITIAL_SIZE = 16;
+
+    private final MultiplexedConnection mConn;
+
+    private ChannelControlBlock[] ccbArray;
+
+    private final BitSet allocationBitmap;
+
+    private final BitSet pendingChannelWriteBitmap;
+
+    private final BitSet pendingChannelCreditsBitmap;
+
+    private final BitSet pendingChannelSynBitmap;
+
+    private final BitSet pendingEOSAckBitmap;
+
+    private int openChannelCount;
+
+    private final IEventCounter pendingWriteEventsCounter;
+
+    ChannelSet(MultiplexedConnection mConn, IEventCounter pendingWriteEventsCounter) {
+        this.mConn = mConn;
+        ccbArray = new ChannelControlBlock[INITIAL_SIZE];
+        allocationBitmap = new BitSet();
+        pendingChannelWriteBitmap = new BitSet();
+        pendingChannelCreditsBitmap = new BitSet();
+        pendingChannelSynBitmap = new BitSet();
+        pendingEOSAckBitmap = new BitSet();
+        this.pendingWriteEventsCounter = pendingWriteEventsCounter;
+        openChannelCount = 0;
+    }
+
+    ChannelControlBlock allocateChannel() throws NetException {
+        synchronized (mConn) {
+            int idx = allocationBitmap.nextClearBit(0);
+            if (idx < 0 || idx == ccbArray.length) {
+                cleanupClosedChannels();
+                idx = allocationBitmap.nextClearBit(0);
+                if (idx < 0 || idx == ccbArray.length) {
+                    idx = ccbArray.length;
+                }
+            }
+            return createChannel(idx);
+        }
+    }
+
+    private void cleanupClosedChannels() {
+        for (int i = 0; i < ccbArray.length; ++i) {
+            ChannelControlBlock ccb = ccbArray[i];
+            if (ccb != null) {
+                if (ccb.completelyClosed()) {
+                    if (LOGGER.isLoggable(Level.FINE)) {
+                        LOGGER.fine("Cleaning free channel: " + ccb);
+                    }
+                    freeChannel(ccb);
+                }
+            }
+        }
+    }
+
+    ChannelControlBlock registerChannel(int channelId) throws NetException {
+        synchronized (mConn) {
+            return createChannel(channelId);
+        }
+    }
+
+    private void freeChannel(ChannelControlBlock channel) {
+        int idx = channel.getChannelId();
+        ccbArray[idx] = null;
+        allocationBitmap.clear(idx);
+        --openChannelCount;
+    }
+
+    ChannelControlBlock getCCB(int channelId) {
+        return ccbArray[channelId];
+    }
+
+    BitSet getPendingChannelWriteBitmap() {
+        return pendingChannelWriteBitmap;
+    }
+
+    BitSet getPendingChannelCreditsBitmap() {
+        return pendingChannelCreditsBitmap;
+    }
+
+    BitSet getPendingChannelSynBitmap() {
+        return pendingChannelSynBitmap;
+    }
+
+    BitSet getPendingEOSAckBitmap() {
+        return pendingEOSAckBitmap;
+    }
+
+    int getOpenChannelCount() {
+        return openChannelCount;
+    }
+
+    void initiateChannelSyn(int channelId) {
+        synchronized (mConn) {
+            assert !pendingChannelSynBitmap.get(channelId);
+            pendingChannelSynBitmap.set(channelId);
+            pendingWriteEventsCounter.increment();
+        }
+    }
+
+    void markPendingCredits(int channelId) {
+        synchronized (mConn) {
+            if (!pendingChannelCreditsBitmap.get(channelId)) {
+                pendingChannelCreditsBitmap.set(channelId);
+                pendingWriteEventsCounter.increment();
+            }
+        }
+    }
+
+    void unmarkPendingCredits(int channelId) {
+        synchronized (mConn) {
+            if (pendingChannelCreditsBitmap.get(channelId)) {
+                pendingChannelCreditsBitmap.clear(channelId);
+                pendingWriteEventsCounter.decrement();
+            }
+        }
+    }
+
+    void markPendingWrite(int channelId) {
+        synchronized (mConn) {
+            assert !pendingChannelWriteBitmap.get(channelId);
+            pendingChannelWriteBitmap.set(channelId);
+            pendingWriteEventsCounter.increment();
+        }
+    }
+
+    void unmarkPendingWrite(int channelId) {
+        synchronized (mConn) {
+            assert pendingChannelWriteBitmap.get(channelId);
+            pendingChannelWriteBitmap.clear(channelId);
+            pendingWriteEventsCounter.decrement();
+        }
+    }
+
+    void markEOSAck(int channelId) {
+        synchronized (mConn) {
+            assert !pendingEOSAckBitmap.get(channelId);
+            pendingEOSAckBitmap.set(channelId);
+            pendingWriteEventsCounter.increment();
+        }
+    }
+
+    private ChannelControlBlock createChannel(int idx) throws NetException {
+        if (idx >= ccbArray.length) {
+            expand(idx);
+        }
+        if (idx > MAX_OPEN_CHANNELS) {
+            throw new NetException("More than " + MAX_OPEN_CHANNELS + " opened concurrently");
+        }
+        if (ccbArray[idx] != null) {
+            assert ccbArray[idx].completelyClosed();
+            if (ccbArray[idx].completelyClosed()) {
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    LOGGER.fine("Cleaning free channel: " + ccbArray[idx]);
+                }
+                freeChannel(ccbArray[idx]);
+            }
+        }
+        assert idx < ccbArray.length;
+        assert !allocationBitmap.get(idx);
+        ChannelControlBlock channel = new ChannelControlBlock(this, idx);
+        ccbArray[idx] = channel;
+        allocationBitmap.set(idx);
+        ++openChannelCount;
+        return channel;
+    }
+
+    private void expand(int idx) {
+        while (idx >= ccbArray.length) {
+            ccbArray = Arrays.copyOf(ccbArray, ccbArray.length * 2);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelOpenListener.java
similarity index 80%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelOpenListener.java
index c728b0b..0fc9b2a 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelOpenListener.java
@@ -12,10 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.protocols.muxdemux;
 
-public interface Accumulator<T, R> {
-    public void accumulate(T o);
-
-    public R getResult();
+public interface IChannelOpenListener {
+    public void channelOpened(ChannelControlBlock channel);
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelReadInterface.java
similarity index 64%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelReadInterface.java
index ec15186..468a617 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelReadInterface.java
@@ -12,12 +12,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.protocols.muxdemux;
 
-import edu.uci.ics.hyracks.control.common.base.INodeController;
+import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
 
-public interface RemoteOp<T> {
-    public String getNodeId();
+public interface IChannelReadInterface {
+    public void setFullBufferAcceptor(ICloseableBufferAcceptor fullBufferAcceptor);
 
-    public T execute(INodeController node) throws Exception;
+    public IBufferAcceptor getEmptyBufferAcceptor();
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelWriteInterface.java
similarity index 63%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelWriteInterface.java
index ec15186..1e53d71 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelWriteInterface.java
@@ -12,12 +12,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.protocols.muxdemux;
 
-import edu.uci.ics.hyracks.control.common.base.INodeController;
+import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
 
-public interface RemoteOp<T> {
-    public String getNodeId();
+public interface IChannelWriteInterface {
+    public void setEmptyBufferAcceptor(IBufferAcceptor emptyBufferAcceptor);
 
-    public T execute(INodeController node) throws Exception;
+    public ICloseableBufferAcceptor getFullBufferAcceptor();
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IEventCounter.java
similarity index 81%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
rename to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IEventCounter.java
index c728b0b..148078c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IEventCounter.java
@@ -12,10 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.protocols.muxdemux;
 
-public interface Accumulator<T, R> {
-    public void accumulate(T o);
+public interface IEventCounter {
+    public void increment();
 
-    public R getResult();
+    public void decrement();
 }
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MultiplexedConnection.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MultiplexedConnection.java
new file mode 100644
index 0000000..1d71fac
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MultiplexedConnection.java
@@ -0,0 +1,353 @@
+/*
+ * 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.net.protocols.muxdemux;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.SocketChannel;
+import java.util.BitSet;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.tcp.ITCPConnectionEventListener;
+import edu.uci.ics.hyracks.net.protocols.tcp.TCPConnection;
+
+public class MultiplexedConnection implements ITCPConnectionEventListener {
+    private static final Logger LOGGER = Logger.getLogger(MultiplexedConnection.class.getName());
+
+    private static final int MAX_CHUNKS_READ_PER_CYCLE = 4;
+
+    private final MuxDemux muxDemux;
+
+    private final IEventCounter pendingWriteEventsCounter;
+
+    private final ChannelSet cSet;
+
+    private final ReaderState readerState;
+
+    private final WriterState writerState;
+
+    private TCPConnection tcpConnection;
+
+    private int lastChannelWritten;
+
+    public MultiplexedConnection(MuxDemux muxDemux) {
+        this.muxDemux = muxDemux;
+        pendingWriteEventsCounter = new IEventCounter() {
+            private int counter;
+
+            @Override
+            public synchronized void increment() {
+                ++counter;
+                if (counter == 1) {
+                    tcpConnection.enable(SelectionKey.OP_WRITE);
+                }
+            }
+
+            @Override
+            public synchronized void decrement() {
+                --counter;
+                if (counter == 0) {
+                    tcpConnection.disable(SelectionKey.OP_WRITE);
+                }
+                if (counter < 0) {
+                    throw new IllegalStateException();
+                }
+            }
+        };
+        cSet = new ChannelSet(this, pendingWriteEventsCounter);
+        readerState = new ReaderState();
+        writerState = new WriterState();
+        lastChannelWritten = -1;
+    }
+
+    synchronized void setTCPConnection(TCPConnection tcpConnection) {
+        this.tcpConnection = tcpConnection;
+        tcpConnection.enable(SelectionKey.OP_READ);
+        notifyAll();
+    }
+
+    synchronized void waitUntilConnected() throws InterruptedException {
+        while (tcpConnection == null) {
+            wait();
+        }
+    }
+
+    @Override
+    public void notifyIOReady(TCPConnection connection, boolean readable, boolean writable) throws IOException,
+            NetException {
+        if (readable) {
+            driveReaderStateMachine();
+        }
+        if (writable) {
+            driveWriterStateMachine();
+        }
+    }
+
+    public ChannelControlBlock openChannel() throws NetException, InterruptedException {
+        ChannelControlBlock channel = cSet.allocateChannel();
+        int channelId = channel.getChannelId();
+        cSet.initiateChannelSyn(channelId);
+        return channel;
+    }
+
+    class WriterState {
+        private final ByteBuffer writeBuffer;
+
+        final MuxDemuxCommand command;
+
+        private ByteBuffer pendingBuffer;
+
+        private int pendingWriteSize;
+
+        private ChannelControlBlock ccb;
+
+        public WriterState() {
+            writeBuffer = ByteBuffer.allocateDirect(MuxDemuxCommand.COMMAND_SIZE);
+            writeBuffer.flip();
+            command = new MuxDemuxCommand();
+            ccb = null;
+        }
+
+        boolean writePending() {
+            return writeBuffer.remaining() > 0 || (pendingBuffer != null && pendingWriteSize > 0);
+        }
+
+        void reset(ByteBuffer pendingBuffer, int pendingWriteSize, ChannelControlBlock ccb) {
+            writeBuffer.clear();
+            command.write(writeBuffer);
+            writeBuffer.flip();
+            this.pendingBuffer = pendingBuffer;
+            this.pendingWriteSize = pendingWriteSize;
+            this.ccb = ccb;
+        }
+
+        boolean performPendingWrite(SocketChannel sc) throws IOException {
+            int len = writeBuffer.remaining();
+            if (len > 0) {
+                int written = sc.write(writeBuffer);
+                muxDemux.getPerformanceCounters().addSignalingBytesWritten(written);
+                if (written < len) {
+                    return false;
+                }
+            }
+            if (pendingBuffer != null) {
+                if (pendingWriteSize > 0) {
+                    assert pendingWriteSize <= pendingBuffer.remaining();
+                    int oldLimit = pendingBuffer.limit();
+                    try {
+                        pendingBuffer.limit(pendingWriteSize + pendingBuffer.position());
+                        int written = sc.write(pendingBuffer);
+                        muxDemux.getPerformanceCounters().addPayloadBytesWritten(written);
+                        pendingWriteSize -= written;
+                    } finally {
+                        pendingBuffer.limit(oldLimit);
+                    }
+                }
+                if (pendingWriteSize > 0) {
+                    return false;
+                }
+                pendingBuffer = null;
+                pendingWriteSize = 0;
+            }
+            if (ccb != null) {
+                ccb.writeComplete();
+                ccb = null;
+            }
+            return true;
+        }
+    }
+
+    void driveWriterStateMachine() throws IOException, NetException {
+        SocketChannel sc = tcpConnection.getSocketChannel();
+        if (writerState.writePending()) {
+            if (!writerState.performPendingWrite(sc)) {
+                return;
+            }
+            pendingWriteEventsCounter.decrement();
+        }
+        int numCycles;
+
+        synchronized (MultiplexedConnection.this) {
+            numCycles = cSet.getOpenChannelCount();
+        }
+
+        for (int i = 0; i < numCycles; ++i) {
+            ChannelControlBlock writeCCB = null;
+            synchronized (MultiplexedConnection.this) {
+                BitSet pendingChannelSynBitmap = cSet.getPendingChannelSynBitmap();
+                for (int j = pendingChannelSynBitmap.nextSetBit(0); j >= 0; j = pendingChannelSynBitmap.nextSetBit(j)) {
+                    pendingChannelSynBitmap.clear(j);
+                    pendingWriteEventsCounter.decrement();
+                    writerState.command.setChannelId(j);
+                    writerState.command.setCommandType(MuxDemuxCommand.CommandType.OPEN_CHANNEL);
+                    writerState.command.setData(0);
+                    writerState.reset(null, 0, null);
+                    if (!writerState.performPendingWrite(sc)) {
+                        return;
+                    }
+                }
+                BitSet pendingChannelCreditsBitmap = cSet.getPendingChannelCreditsBitmap();
+                for (int j = pendingChannelCreditsBitmap.nextSetBit(0); j >= 0; j = pendingChannelCreditsBitmap
+                        .nextSetBit(j)) {
+                    pendingChannelCreditsBitmap.clear(j);
+                    pendingWriteEventsCounter.decrement();
+                    writerState.command.setChannelId(j);
+                    writerState.command.setCommandType(MuxDemuxCommand.CommandType.ADD_CREDITS);
+                    ChannelControlBlock ccb = cSet.getCCB(j);
+                    int credits = ccb.getAndResetReadCredits();
+                    writerState.command.setData(credits);
+                    writerState.reset(null, 0, null);
+                    if (!writerState.performPendingWrite(sc)) {
+                        return;
+                    }
+                }
+                BitSet pendingEOSAckBitmap = cSet.getPendingEOSAckBitmap();
+                for (int j = pendingEOSAckBitmap.nextSetBit(0); j >= 0; j = pendingEOSAckBitmap.nextSetBit(j)) {
+                    pendingEOSAckBitmap.clear(j);
+                    pendingWriteEventsCounter.decrement();
+                    writerState.command.setChannelId(j);
+                    writerState.command.setCommandType(MuxDemuxCommand.CommandType.CLOSE_CHANNEL_ACK);
+                    writerState.command.setData(0);
+                    writerState.reset(null, 0, null);
+                    if (!writerState.performPendingWrite(sc)) {
+                        return;
+                    }
+                }
+                BitSet pendingChannelWriteBitmap = cSet.getPendingChannelWriteBitmap();
+                lastChannelWritten = pendingChannelWriteBitmap.nextSetBit(lastChannelWritten + 1);
+                if (lastChannelWritten < 0) {
+                    lastChannelWritten = pendingChannelWriteBitmap.nextSetBit(0);
+                    if (lastChannelWritten < 0) {
+                        return;
+                    }
+                }
+                writeCCB = cSet.getCCB(lastChannelWritten);
+            }
+            writeCCB.write(writerState);
+            if (writerState.writePending()) {
+                pendingWriteEventsCounter.increment();
+                if (!writerState.performPendingWrite(sc)) {
+                    return;
+                }
+                pendingWriteEventsCounter.decrement();
+            }
+        }
+    }
+
+    class ReaderState {
+        private final ByteBuffer readBuffer;
+
+        final MuxDemuxCommand command;
+
+        private int pendingReadSize;
+
+        private ChannelControlBlock ccb;
+
+        ReaderState() {
+            readBuffer = ByteBuffer.allocateDirect(MuxDemuxCommand.COMMAND_SIZE);
+            command = new MuxDemuxCommand();
+        }
+
+        void reset() {
+            readBuffer.clear();
+            pendingReadSize = 0;
+            ccb = null;
+        }
+
+        private ChannelControlBlock getCCBInCommand() {
+            synchronized (MultiplexedConnection.this) {
+                return cSet.getCCB(command.getChannelId());
+            }
+        }
+    }
+
+    void driveReaderStateMachine() throws IOException, NetException {
+        SocketChannel sc = tcpConnection.getSocketChannel();
+        int chunksRead = 0;
+        while (chunksRead < MAX_CHUNKS_READ_PER_CYCLE) {
+            if (readerState.readBuffer.remaining() > 0) {
+                int read = sc.read(readerState.readBuffer);
+                if (read < 0) {
+                    throw new NetException("Socket Closed");
+                }
+                muxDemux.getPerformanceCounters().addSignalingBytesRead(read);
+                if (readerState.readBuffer.remaining() > 0) {
+                    return;
+                }
+                readerState.readBuffer.flip();
+                readerState.command.read(readerState.readBuffer);
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    LOGGER.fine("Received command: " + readerState.command);
+                }
+                ChannelControlBlock ccb = null;
+                switch (readerState.command.getCommandType()) {
+                    case ADD_CREDITS: {
+                        ccb = readerState.getCCBInCommand();
+                        ccb.addWriteCredits(readerState.command.getData());
+                        break;
+                    }
+                    case CLOSE_CHANNEL: {
+                        ccb = readerState.getCCBInCommand();
+                        ccb.reportRemoteEOS();
+                        int channelId = ccb.getChannelId();
+                        cSet.markEOSAck(channelId);
+                        cSet.unmarkPendingCredits(channelId);
+                        break;
+                    }
+                    case CLOSE_CHANNEL_ACK: {
+                        ccb = readerState.getCCBInCommand();
+                        ccb.reportLocalEOSAck();
+                        break;
+                    }
+                    case DATA: {
+                        ccb = readerState.getCCBInCommand();
+                        readerState.pendingReadSize = readerState.command.getData();
+                        readerState.ccb = ccb;
+                        break;
+                    }
+                    case ERROR: {
+                        ccb = readerState.getCCBInCommand();
+                        ccb.reportRemoteError(readerState.command.getData());
+                        int channelId = ccb.getChannelId();
+                        cSet.markEOSAck(channelId);
+                        cSet.unmarkPendingCredits(channelId);
+                        break;
+                    }
+                    case OPEN_CHANNEL: {
+                        int channelId = readerState.command.getChannelId();
+                        ccb = cSet.registerChannel(channelId);
+                        muxDemux.getChannelOpenListener().channelOpened(ccb);
+                    }
+                }
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    LOGGER.fine("Applied command: " + readerState.command + " on " + ccb);
+                }
+            }
+            if (readerState.pendingReadSize > 0) {
+                ++chunksRead;
+                int newPendingReadSize = readerState.ccb.read(sc, readerState.pendingReadSize);
+                muxDemux.getPerformanceCounters().addPayloadBytesRead(readerState.pendingReadSize - newPendingReadSize);
+                readerState.pendingReadSize = newPendingReadSize;
+                if (readerState.pendingReadSize > 0) {
+                    return;
+                }
+            }
+            readerState.reset();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
new file mode 100644
index 0000000..b041e2c
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
@@ -0,0 +1,94 @@
+/*
+ * 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.net.protocols.muxdemux;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.net.protocols.tcp.ITCPConnectionListener;
+import edu.uci.ics.hyracks.net.protocols.tcp.TCPConnection;
+import edu.uci.ics.hyracks.net.protocols.tcp.TCPEndpoint;
+
+public class MuxDemux {
+    private final InetSocketAddress localAddress;
+
+    private final IChannelOpenListener channelOpenListener;
+
+    private final Map<InetSocketAddress, MultiplexedConnection> connectionMap;
+
+    private final TCPEndpoint tcpEndpoint;
+
+    private final PerformanceCounters perfCounters;
+
+    public MuxDemux(InetSocketAddress localAddress, IChannelOpenListener listener, int nThreads) {
+        this.localAddress = localAddress;
+        this.channelOpenListener = listener;
+        connectionMap = new HashMap<InetSocketAddress, MultiplexedConnection>();
+        this.tcpEndpoint = new TCPEndpoint(new ITCPConnectionListener() {
+            @Override
+            public void connectionEstablished(TCPConnection connection) {
+                MultiplexedConnection mConn;
+                synchronized (MuxDemux.this) {
+                    mConn = connectionMap.get(connection.getRemoteAddress());
+                }
+                assert mConn != null;
+                mConn.setTCPConnection(connection);
+                connection.setEventListener(mConn);
+                connection.setAttachment(mConn);
+            }
+
+            @Override
+            public void acceptedConnection(TCPConnection connection) {
+                MultiplexedConnection mConn = new MultiplexedConnection(MuxDemux.this);
+                mConn.setTCPConnection(connection);
+                connection.setEventListener(mConn);
+                connection.setAttachment(mConn);
+            }
+        }, nThreads);
+        perfCounters = new PerformanceCounters();
+    }
+
+    public void start() throws IOException {
+        tcpEndpoint.start(localAddress);
+    }
+
+    public MultiplexedConnection connect(InetSocketAddress remoteAddress) throws InterruptedException {
+        MultiplexedConnection mConn = null;
+        synchronized (this) {
+            mConn = connectionMap.get(remoteAddress);
+            if (mConn == null) {
+                mConn = new MultiplexedConnection(this);
+                connectionMap.put(remoteAddress, mConn);
+                tcpEndpoint.initiateConnection(remoteAddress);
+            }
+        }
+        mConn.waitUntilConnected();
+        return mConn;
+    }
+
+    IChannelOpenListener getChannelOpenListener() {
+        return channelOpenListener;
+    }
+
+    public InetSocketAddress getLocalAddress() {
+        return tcpEndpoint.getLocalAddress();
+    }
+
+    public PerformanceCounters getPerformanceCounters() {
+        return perfCounters;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemuxCommand.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemuxCommand.java
new file mode 100644
index 0000000..9124aad
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemuxCommand.java
@@ -0,0 +1,75 @@
+package edu.uci.ics.hyracks.net.protocols.muxdemux;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+
+class MuxDemuxCommand {
+    static final int MAX_CHANNEL_ID = 0x3ff;
+
+    static final int COMMAND_SIZE = 4;
+
+    static final int MAX_DATA_VALUE = 0x7ffff;
+
+    enum CommandType {
+        OPEN_CHANNEL,
+        CLOSE_CHANNEL,
+        CLOSE_CHANNEL_ACK,
+        ERROR,
+        ADD_CREDITS,
+        DATA,
+    }
+
+    private int channelId;
+
+    private CommandType type;
+
+    private int data;
+
+    public int getChannelId() {
+        return channelId;
+    }
+
+    public void setChannelId(int channelId) throws NetException {
+        if (channelId > MAX_CHANNEL_ID) {
+            throw new NetException("channelId " + channelId + " exceeds " + MAX_CHANNEL_ID);
+        }
+        this.channelId = channelId;
+    }
+
+    public CommandType getCommandType() {
+        return type;
+    }
+
+    public void setCommandType(CommandType type) {
+        this.type = type;
+    }
+
+    public int getData() {
+        return data;
+    }
+
+    public void setData(int data) throws NetException {
+        if (channelId > MAX_DATA_VALUE) {
+            throw new NetException("data " + data + " exceeds " + MAX_DATA_VALUE);
+        }
+        this.data = data;
+    }
+
+    public void write(ByteBuffer buffer) {
+        int cmd = (channelId << 22) | (type.ordinal() << 19) | (data & 0x7ffff);
+        buffer.putInt(cmd);
+    }
+
+    public void read(ByteBuffer buffer) {
+        int cmd = buffer.getInt();
+        channelId = (cmd >> 22) & 0x3ff;
+        type = CommandType.values()[(cmd >> 19) & 0x7];
+        data = cmd & 0x7ffff;
+    }
+
+    @Override
+    public String toString() {
+        return channelId + ":" + type + ":" + data;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/PerformanceCounters.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/PerformanceCounters.java
new file mode 100644
index 0000000..a203f06
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/PerformanceCounters.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.net.protocols.muxdemux;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+public class PerformanceCounters {
+    private final AtomicLong payloadBytesRead;
+
+    private final AtomicLong payloadBytesWritten;
+
+    private final AtomicLong signalingBytesRead;
+
+    private final AtomicLong signalingBytesWritten;
+
+    public PerformanceCounters() {
+        payloadBytesRead = new AtomicLong();
+        payloadBytesWritten = new AtomicLong();
+        signalingBytesRead = new AtomicLong();
+        signalingBytesWritten = new AtomicLong();
+    }
+
+    public void addPayloadBytesRead(long delta) {
+        payloadBytesRead.addAndGet(delta);
+    }
+
+    public long getPayloadBytesRead() {
+        return payloadBytesRead.get();
+    }
+
+    public void addPayloadBytesWritten(long delta) {
+        payloadBytesWritten.addAndGet(delta);
+    }
+
+    public long getPayloadBytesWritten() {
+        return payloadBytesWritten.get();
+    }
+
+    public void addSignalingBytesRead(long delta) {
+        signalingBytesRead.addAndGet(delta);
+    }
+
+    public long getSignalingBytesRead() {
+        return signalingBytesRead.get();
+    }
+
+    public void addSignalingBytesWritten(long delta) {
+        signalingBytesWritten.addAndGet(delta);
+    }
+
+    public long getSignalingBytesWritten() {
+        return signalingBytesWritten.get();
+    }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/ITCPConnectionEventListener.java
similarity index 66%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
rename to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/ITCPConnectionEventListener.java
index ec15186..607bf31 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/ITCPConnectionEventListener.java
@@ -12,12 +12,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.protocols.tcp;
 
-import edu.uci.ics.hyracks.control.common.base.INodeController;
+import java.io.IOException;
 
-public interface RemoteOp<T> {
-    public String getNodeId();
+import edu.uci.ics.hyracks.net.exceptions.NetException;
 
-    public T execute(INodeController node) throws Exception;
+public interface ITCPConnectionEventListener {
+    public void notifyIOReady(TCPConnection connection, boolean readable, boolean writable) throws IOException,
+            NetException;
 }
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/ITCPConnectionListener.java
similarity index 74%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
copy to hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/ITCPConnectionListener.java
index c728b0b..cdaabf4 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/Accumulator.java
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/ITCPConnectionListener.java
@@ -12,10 +12,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.remote;
+package edu.uci.ics.hyracks.net.protocols.tcp;
 
-public interface Accumulator<T, R> {
-    public void accumulate(T o);
+public interface ITCPConnectionListener {
+    public void acceptedConnection(TCPConnection connection);
 
-    public R getResult();
+    public void connectionEstablished(TCPConnection connection);
 }
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPConnection.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPConnection.java
new file mode 100644
index 0000000..210508b
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPConnection.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.net.protocols.tcp;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.SocketChannel;
+
+public class TCPConnection {
+    private final TCPEndpoint endpoint;
+
+    private final SocketChannel channel;
+
+    private final SelectionKey key;
+
+    private final Selector selector;
+
+    private ITCPConnectionEventListener eventListener;
+
+    private Object attachment;
+
+    public TCPConnection(TCPEndpoint endpoint, SocketChannel channel, SelectionKey key, Selector selector) {
+        this.endpoint = endpoint;
+        this.channel = channel;
+        this.key = key;
+        this.selector = selector;
+    }
+
+    public TCPEndpoint getEndpoint() {
+        return endpoint;
+    }
+
+    public SocketChannel getSocketChannel() {
+        return channel;
+    }
+
+    public InetSocketAddress getLocalAddress() {
+        return (InetSocketAddress) channel.socket().getLocalSocketAddress();
+    }
+
+    public InetSocketAddress getRemoteAddress() {
+        return (InetSocketAddress) channel.socket().getRemoteSocketAddress();
+    }
+
+    public void enable(int ops) {
+        key.interestOps(key.interestOps() | ops);
+        selector.wakeup();
+    }
+
+    public void disable(int ops) {
+        key.interestOps(key.interestOps() & ~(ops));
+        selector.wakeup();
+    }
+
+    public ITCPConnectionEventListener getEventListener() {
+        return eventListener;
+    }
+
+    public void setEventListener(ITCPConnectionEventListener eventListener) {
+        this.eventListener = eventListener;
+    }
+
+    public Object getAttachment() {
+        return attachment;
+    }
+
+    public void setAttachment(Object attachment) {
+        this.attachment = attachment;
+    }
+
+    public void close() {
+        key.cancel();
+        try {
+            channel.close();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
new file mode 100644
index 0000000..d73ba21
--- /dev/null
+++ b/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
@@ -0,0 +1,198 @@
+/*
+ * 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.net.protocols.tcp;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.nio.channels.SelectableChannel;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public class TCPEndpoint {
+    private final ITCPConnectionListener connectionListener;
+
+    private final int nThreads;
+
+    private ServerSocketChannel serverSocketChannel;
+
+    private InetSocketAddress localAddress;
+
+    private IOThread[] ioThreads;
+
+    private int nextThread;
+
+    public TCPEndpoint(ITCPConnectionListener connectionListener, int nThreads) {
+        this.connectionListener = connectionListener;
+        this.nThreads = nThreads;
+    }
+
+    public void start(InetSocketAddress localAddress) throws IOException {
+        serverSocketChannel = ServerSocketChannel.open();
+        ServerSocket serverSocket = serverSocketChannel.socket();
+        serverSocket.bind(localAddress);
+        this.localAddress = (InetSocketAddress) serverSocket.getLocalSocketAddress();
+        ioThreads = new IOThread[nThreads];
+        for (int i = 0; i < ioThreads.length; ++i) {
+            ioThreads[i] = new IOThread();
+        }
+        ioThreads[0].registerServerSocket(serverSocketChannel);
+        for (int i = 0; i < ioThreads.length; ++i) {
+            ioThreads[i].start();
+        }
+    }
+
+    private synchronized int getNextThread() {
+        int result = nextThread;
+        nextThread = (nextThread + 1) % nThreads;
+        return result;
+    }
+
+    public void initiateConnection(InetSocketAddress remoteAddress) {
+        int targetThread = getNextThread();
+        ioThreads[targetThread].initiateConnection(remoteAddress);
+    }
+
+    private void distributeIncomingConnection(SocketChannel channel) {
+        int targetThread = getNextThread();
+        ioThreads[targetThread].addIncomingConnection(channel);
+    }
+
+    public InetSocketAddress getLocalAddress() {
+        return localAddress;
+    }
+
+    private class IOThread extends Thread {
+        private final List<InetSocketAddress> pendingConnections;
+
+        private final List<InetSocketAddress> workingPendingConnections;
+
+        private final List<SocketChannel> incomingConnections;
+
+        private final List<SocketChannel> workingIncomingConnections;
+
+        private Selector selector;
+
+        public IOThread() throws IOException {
+            super("TCPEndpoint IO Thread");
+            setPriority(MAX_PRIORITY);
+            this.pendingConnections = new ArrayList<InetSocketAddress>();
+            this.workingPendingConnections = new ArrayList<InetSocketAddress>();
+            this.incomingConnections = new ArrayList<SocketChannel>();
+            this.workingIncomingConnections = new ArrayList<SocketChannel>();
+            selector = Selector.open();
+        }
+
+        @Override
+        public void run() {
+            while (true) {
+                try {
+                    int n = selector.select();
+                    collectOutstandingWork();
+                    if (!workingPendingConnections.isEmpty()) {
+                        for (InetSocketAddress address : workingPendingConnections) {
+                            SocketChannel channel = SocketChannel.open();
+                            channel.configureBlocking(false);
+                            if (!channel.connect(address)) {
+                                channel.register(selector, SelectionKey.OP_CONNECT);
+                            } else {
+                                SelectionKey key = channel.register(selector, 0);
+                                createConnection(key, channel);
+                            }
+                        }
+                        workingPendingConnections.clear();
+                    }
+                    if (!workingIncomingConnections.isEmpty()) {
+                        for (SocketChannel channel : workingIncomingConnections) {
+                            channel.configureBlocking(false);
+                            SelectionKey sKey = channel.register(selector, 0);
+                            TCPConnection connection = new TCPConnection(TCPEndpoint.this, channel, sKey, selector);
+                            sKey.attach(connection);
+                            synchronized (connectionListener) {
+                                connectionListener.acceptedConnection(connection);
+                            }
+                        }
+                        workingIncomingConnections.clear();
+                    }
+                    if (n > 0) {
+                        Iterator<SelectionKey> i = selector.selectedKeys().iterator();
+                        while (i.hasNext()) {
+                            SelectionKey key = i.next();
+                            i.remove();
+                            SelectableChannel sc = key.channel();
+                            boolean readable = key.isReadable();
+                            boolean writable = key.isWritable();
+
+                            if (readable || writable) {
+                                TCPConnection connection = (TCPConnection) key.attachment();
+                                connection.getEventListener().notifyIOReady(connection, readable, writable);
+                            }
+                            if (key.isAcceptable()) {
+                                assert sc == serverSocketChannel;
+                                SocketChannel channel = serverSocketChannel.accept();
+                                distributeIncomingConnection(channel);
+                            } else if (key.isConnectable()) {
+                                SocketChannel channel = (SocketChannel) sc;
+                                if (channel.finishConnect()) {
+                                    createConnection(key, channel);
+                                }
+                            }
+                        }
+                    }
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+
+        private void createConnection(SelectionKey key, SocketChannel channel) {
+            TCPConnection connection = new TCPConnection(TCPEndpoint.this, channel, key, selector);
+            key.attach(connection);
+            key.interestOps(0);
+            connectionListener.connectionEstablished(connection);
+        }
+
+        synchronized void initiateConnection(InetSocketAddress remoteAddress) {
+            pendingConnections.add(remoteAddress);
+            selector.wakeup();
+        }
+
+        synchronized void addIncomingConnection(SocketChannel channel) {
+            incomingConnections.add(channel);
+            selector.wakeup();
+        }
+
+        void registerServerSocket(ServerSocketChannel serverSocketChannel) throws IOException {
+            serverSocketChannel.configureBlocking(false);
+            serverSocketChannel.register(selector, SelectionKey.OP_ACCEPT);
+        }
+
+        private synchronized void collectOutstandingWork() {
+            if (!pendingConnections.isEmpty()) {
+                workingPendingConnections.addAll(pendingConnections);
+                pendingConnections.clear();
+            }
+            if (!incomingConnections.isEmpty()) {
+                workingIncomingConnections.addAll(incomingConnections);
+                incomingConnections.clear();
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-net/src/test/java/edu/uci/ics/hyracks/net/tests/NetTest.java b/hyracks-net/src/test/java/edu/uci/ics/hyracks/net/tests/NetTest.java
new file mode 100644
index 0000000..31bc2df
--- /dev/null
+++ b/hyracks-net/src/test/java/edu/uci/ics/hyracks/net/tests/NetTest.java
@@ -0,0 +1,213 @@
+/*
+ * 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.net.tests;
+
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.IChannelOpenListener;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
+
+public class NetTest {
+    @Test
+    public void test() throws Exception {
+        AtomicBoolean failFlag = new AtomicBoolean();
+
+        MuxDemux md1 = createMuxDemux("md1", failFlag);
+        md1.start();
+        MuxDemux md2 = createMuxDemux("md2", failFlag);
+        md2.start();
+        InetSocketAddress md2Address = md2.getLocalAddress();
+
+        MultiplexedConnection md1md2 = md1.connect(md2Address);
+
+        Thread t1 = createThread(md1md2, 1);
+        Thread t2 = createThread(md1md2, -1);
+        t1.start();
+        t2.start();
+
+        t1.join();
+        t2.join();
+
+        Assert.assertFalse("Failure flag was set to true", failFlag.get());
+    }
+
+    private Thread createThread(final MultiplexedConnection md1md2, final int factor) {
+        return new Thread() {
+            @Override
+            public void run() {
+                try {
+                    ChannelControlBlock md1md2c1 = md1md2.openChannel();
+
+                    final Semaphore sem = new Semaphore(1);
+                    sem.acquire();
+                    md1md2c1.getWriteInterface().setEmptyBufferAcceptor(new IBufferAcceptor() {
+                        @Override
+                        public void accept(ByteBuffer buffer) {
+                        }
+                    });
+
+                    md1md2c1.getReadInterface().setFullBufferAcceptor(new ICloseableBufferAcceptor() {
+                        @Override
+                        public void accept(ByteBuffer buffer) {
+                        }
+
+                        @Override
+                        public void error(int ecode) {
+                        }
+
+                        @Override
+                        public void close() {
+                            sem.release();
+                        }
+                    });
+
+                    ICloseableBufferAcceptor fba = md1md2c1.getWriteInterface().getFullBufferAcceptor();
+                    for (int i = 0; i < 10000; ++i) {
+                        ByteBuffer buffer = ByteBuffer.allocate(1024);
+                        for (int j = 0; j < 256; ++j) {
+                            buffer.putInt(factor * (i + j));
+                        }
+                        buffer.flip();
+                        fba.accept(buffer);
+                    }
+                    fba.close();
+                    sem.acquire();
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        };
+
+    }
+
+    private MuxDemux createMuxDemux(final String label, final AtomicBoolean failFlag) {
+        IChannelOpenListener md1OpenListener = new IChannelOpenListener() {
+            @Override
+            public void channelOpened(final ChannelControlBlock channel) {
+                final ChannelIO cio = new ChannelIO(label, channel);
+                channel.getReadInterface().setFullBufferAcceptor(cio.rifba);
+                channel.getWriteInterface().setEmptyBufferAcceptor(cio.wieba);
+
+                final IBufferAcceptor rieba = channel.getReadInterface().getEmptyBufferAcceptor();
+                for (int i = 0; i < 50; ++i) {
+                    rieba.accept(ByteBuffer.allocate(1024));
+                }
+                new Thread() {
+                    private int prevTotal = 0;
+
+                    @Override
+                    public void run() {
+                        while (true) {
+                            ByteBuffer fbuf = null;
+                            synchronized (channel) {
+                                while (!cio.eos && cio.ecode == 0 && cio.rifq.isEmpty()) {
+                                    try {
+                                        channel.wait();
+                                    } catch (InterruptedException e) {
+                                        e.printStackTrace();
+                                    }
+                                }
+                                if (!cio.rifq.isEmpty()) {
+                                    fbuf = cio.rifq.poll();
+                                } else if (cio.ecode != 0) {
+                                    throw new RuntimeException("Error: " + cio.ecode);
+                                } else if (cio.eos) {
+                                    channel.getWriteInterface().getFullBufferAcceptor().close();
+                                    return;
+                                }
+                            }
+                            int counter = 0;
+                            while (fbuf.remaining() > 0) {
+                                counter += fbuf.getInt();
+                            }
+                            if (prevTotal != 0) {
+                                if (Math.abs(counter - prevTotal) != 256) {
+                                    failFlag.set(true);
+                                }
+                            }
+                            prevTotal = counter;
+                            fbuf.compact();
+                            rieba.accept(fbuf);
+                        }
+                    }
+                }.start();
+            }
+        };
+        return new MuxDemux(new InetSocketAddress("127.0.0.1", 0), md1OpenListener, 1);
+    }
+
+    private class ChannelIO {
+        private ChannelControlBlock channel;
+
+        private Queue<ByteBuffer> rifq;
+
+        private Queue<ByteBuffer> wieq;
+
+        private boolean eos;
+
+        private int ecode;
+
+        private ICloseableBufferAcceptor rifba;
+
+        private IBufferAcceptor wieba;
+
+        public ChannelIO(final String label, ChannelControlBlock channel) {
+            this.channel = channel;
+            this.rifq = new LinkedList<ByteBuffer>();
+            this.wieq = new LinkedList<ByteBuffer>();
+
+            rifba = new ICloseableBufferAcceptor() {
+                @Override
+                public void accept(ByteBuffer buffer) {
+                    rifq.add(buffer);
+                    ChannelIO.this.channel.notifyAll();
+                }
+
+                @Override
+                public void error(int ecode) {
+                    ChannelIO.this.ecode = ecode;
+                    ChannelIO.this.channel.notifyAll();
+                }
+
+                @Override
+                public void close() {
+                    eos = true;
+                    ChannelIO.this.channel.notifyAll();
+                }
+            };
+
+            wieba = new IBufferAcceptor() {
+                @Override
+                public void accept(ByteBuffer buffer) {
+                    wieq.add(buffer);
+                    ChannelIO.this.channel.notifyAll();
+                }
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index 890f42d..0ab0646 100644
--- a/pom.xml
+++ b/pom.xml
@@ -25,7 +25,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
             <forkMode>pertest</forkMode>
-            <argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties</argLine>
+            <argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties -Xdebug -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n</argLine>
         </configuration>
       </plugin>
     </plugins>
@@ -86,6 +86,7 @@
     <module>hyracks-dataflow-hadoop</module>
     <module>hyracks-control-common</module>
     <module>hyracks-control-cc</module>
+    <module>hyracks-net</module>
     <module>hyracks-control-nc</module>
     <module>hyracks-data</module>
     <module>hyracks-cli</module>