Implements concurrent query management support.

The following changes are included:
-- factor out JobManager, NodeManager, and ResourceManager from ClusterControllerService;
-- let each application plugin its own IJobCapacityController implementation;
-- let each job specify its required cluster capacity;
-- add a required cluster capacity estimation visitor for optimized query plans;
-- add admission control and queuing for queries, but always executes DDLs and DMLs immediately;
-- add tests for JobManger, NodeManager, ClusterCapacity, ClusterCapacityVisitor, and IJobCapacityController;
-- enlarge the -Xmx setting for ManagixSqlppExecutionTest.

Change-Id: I8fb6fda57efa139114dd234e08cc7de7129468c8
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1424
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
BAD: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <tillw@apache.org>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 90a8599..c4535cf 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -61,9 +61,10 @@
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.asterix.runtime.util.AppContextInfo;
 import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
-import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
+import org.apache.asterix.translator.SessionConfig;
+import org.apache.asterix.util.ResourceUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -89,6 +90,7 @@
 import org.apache.hyracks.api.client.IClusterInfoCollector;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.client.NodeControllerInfo;
+import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 
@@ -238,8 +240,9 @@
 
         int parallelism = getParallelism(querySpecificConfig.get(CompilerProperties.COMPILER_PARALLELISM_KEY),
                 compilerProperties.getParallelism());
-        builder.setClusterLocations(parallelism == CompilerProperties.COMPILER_PARALLELISM_AS_STORAGE
-                ? metadataProvider.getClusterLocations() : getComputationLocations(clusterInfoCollector, parallelism));
+        AlgebricksAbsolutePartitionConstraint computationLocations = chooseLocations(clusterInfoCollector, parallelism,
+                metadataProvider.getClusterLocations());
+        builder.setClusterLocations(computationLocations);
 
         ICompiler compiler = compilerFactory.createCompiler(plan, metadataProvider, t.getVarCounter());
         if (conf.isOptimize()) {
@@ -314,6 +317,14 @@
                 metadataProvider.isWriteTransaction());
         JobSpecification spec = compiler.createJob(AppContextInfo.INSTANCE, jobEventListenerFactory);
 
+        // When the top-level statement is a query, the statement parameter is null.
+        if (statement == null) {
+            // Sets a required capacity, only for read-only queries.
+            // DDLs and DMLs are considered not that frequent.
+            spec.setRequiredClusterCapacity(ResourceUtils.getRequiredCompacity(plan, computationLocations,
+                    sortFrameLimit, groupFrameLimit, joinFrameLimit, frameSize));
+        }
+
         if (conf.is(SessionConfig.OOB_HYRACKS_JOB)) {
             printPlanPrefix(conf, "Hyracks job");
             if (rwQ != null) {
@@ -364,54 +375,78 @@
         }
     }
 
-    // Computes the location constraints based on user-configured parallelism parameter.
-    // Note that the parallelism parameter is only a hint -- it will not be respected if it is too small or too large.
-    private AlgebricksAbsolutePartitionConstraint getComputationLocations(IClusterInfoCollector clusterInfoCollector,
-            int parallelismHint) throws AlgebricksException {
+    // Chooses the location constraints, i.e., whether to use storage parallelism or use a user-sepcified number
+    // of cores.
+    private AlgebricksAbsolutePartitionConstraint chooseLocations(IClusterInfoCollector clusterInfoCollector,
+            int parallelismHint, AlgebricksAbsolutePartitionConstraint storageLocations) throws AlgebricksException {
         try {
             Map<String, NodeControllerInfo> ncMap = clusterInfoCollector.getNodeControllerInfos();
 
-            // Unifies the handling of non-positive parallelism.
-            int parallelism = parallelismHint <= 0 ? -2 * ncMap.size() : parallelismHint;
+            // Gets total number of cores in the cluster.
+            int totalNumCores = getTotalNumCores(ncMap);
 
-            // Calculates per node parallelism, with load balance, i.e., randomly selecting nodes with larger
-            // parallelism.
-            int numNodes = ncMap.size();
-            int numNodesWithOneMorePartition = parallelism % numNodes;
-            int perNodeParallelismMin = parallelism / numNodes;
-            int perNodeParallelismMax = parallelism / numNodes + 1;
-            List<String> allNodes = new ArrayList<>();
-            Set<String> selectedNodesWithOneMorePartition = new HashSet<>();
-            for (Map.Entry<String, NodeControllerInfo> entry : ncMap.entrySet()) {
-                allNodes.add(entry.getKey());
+            // If storage parallelism is not larger than the total number of cores, we use the storage parallelism.
+            // Otherwise, we will use all available cores.
+            if (parallelismHint == CompilerProperties.COMPILER_PARALLELISM_AS_STORAGE
+                    && storageLocations.getLocations().length <= totalNumCores) {
+                return storageLocations;
             }
-            Random random = new Random();
-            for (int index = numNodesWithOneMorePartition; index >= 1; --index) {
-                int pick = random.nextInt(index);
-                selectedNodesWithOneMorePartition.add(allNodes.get(pick));
-                Collections.swap(allNodes, pick, index - 1);
-            }
-
-            // Generates cluster locations, which has duplicates for a node if it contains more than one partitions.
-            List<String> locations = new ArrayList<>();
-            for (Map.Entry<String, NodeControllerInfo> entry : ncMap.entrySet()) {
-                String nodeId = entry.getKey();
-                int numCores = entry.getValue().getNumCores();
-                int availableCores = numCores > 1 ? numCores - 1 : numCores; // Reserves one core for heartbeat.
-                int nodeParallelism = selectedNodesWithOneMorePartition.contains(nodeId) ? perNodeParallelismMax
-                        : perNodeParallelismMin;
-                int coresToUse = nodeParallelism >= 0 && nodeParallelism < availableCores ? nodeParallelism
-                        : availableCores;
-                for (int count = 0; count < coresToUse; ++count) {
-                    locations.add(nodeId);
-                }
-            }
-            return new AlgebricksAbsolutePartitionConstraint(locations.toArray(new String[0]));
-        } catch (Exception e) {
+            return getComputationLocations(ncMap, parallelismHint);
+        } catch (HyracksException e) {
             throw new AlgebricksException(e);
         }
     }
 
+    // Computes the location constraints based on user-configured parallelism parameter.
+    // Note that the parallelism parameter is only a hint -- it will not be respected if it is too small or too large.
+    private AlgebricksAbsolutePartitionConstraint getComputationLocations(Map<String, NodeControllerInfo> ncMap,
+            int parallelismHint) {
+        // Unifies the handling of non-positive parallelism.
+        int parallelism = parallelismHint <= 0 ? -2 * ncMap.size() : parallelismHint;
+
+        // Calculates per node parallelism, with load balance, i.e., randomly selecting nodes with larger
+        // parallelism.
+        int numNodes = ncMap.size();
+        int numNodesWithOneMorePartition = parallelism % numNodes;
+        int perNodeParallelismMin = parallelism / numNodes;
+        int perNodeParallelismMax = parallelism / numNodes + 1;
+        List<String> allNodes = new ArrayList<>();
+        Set<String> selectedNodesWithOneMorePartition = new HashSet<>();
+        for (Map.Entry<String, NodeControllerInfo> entry : ncMap.entrySet()) {
+            allNodes.add(entry.getKey());
+        }
+        Random random = new Random();
+        for (int index = numNodesWithOneMorePartition; index >= 1; --index) {
+            int pick = random.nextInt(index);
+            selectedNodesWithOneMorePartition.add(allNodes.get(pick));
+            Collections.swap(allNodes, pick, index - 1);
+        }
+
+        // Generates cluster locations, which has duplicates for a node if it contains more than one partitions.
+        List<String> locations = new ArrayList<>();
+        for (Map.Entry<String, NodeControllerInfo> entry : ncMap.entrySet()) {
+            String nodeId = entry.getKey();
+            int availableCores = entry.getValue().getNumAvailableCores();
+            int nodeParallelism = selectedNodesWithOneMorePartition.contains(nodeId) ? perNodeParallelismMax
+                    : perNodeParallelismMin;
+            int coresToUse = nodeParallelism >= 0 && nodeParallelism < availableCores ? nodeParallelism
+                    : availableCores;
+            for (int count = 0; count < coresToUse; ++count) {
+                locations.add(nodeId);
+            }
+        }
+        return new AlgebricksAbsolutePartitionConstraint(locations.toArray(new String[0]));
+    }
+
+    // Gets the total number of available cores in the cluster.
+    private int getTotalNumCores(Map<String, NodeControllerInfo> ncMap) {
+        int sum = 0;
+        for (Map.Entry<String, NodeControllerInfo> entry : ncMap.entrySet()) {
+            sum += entry.getValue().getNumAvailableCores();
+        }
+        return sum;
+    }
+
     // Gets the frame limit.
     private int getFrameLimit(String parameter, long memBudgetInConfiguration, int frameSize) {
         IPropertyInterpreter<Long> longBytePropertyInterpreter = PropertyInterpreters.getLongBytePropertyInterpreter();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
new file mode 100644
index 0000000..3a6bfee
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.app.resource;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteUpsertOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SplitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import org.apache.hyracks.api.job.resource.IClusterCapacity;
+
+// The current implementation aggregates the memory requirement for each operator.
+// TODO(buyingyi): consider stages for calculating the memory requirement.
+public class RequiredCapacityVisitor implements ILogicalOperatorVisitor<Void, Void> {
+
+    private static final long MAX_BUFFER_PER_CONNECTION = 1L;
+
+    private final long numComputationPartitions;
+    private final long groupByMemorySize;
+    private final long joinMemorySize;
+    private final long sortMemorySize;
+    private final long frameSize;
+    private final IClusterCapacity clusterCapacity;
+    private final Set<ILogicalOperator> visitedOperators = new HashSet<>();
+    private long stageMemorySoFar = 0L;
+
+    public RequiredCapacityVisitor(int numComputationPartitions, int sortFrameLimit, int groupFrameLimit,
+            int joinFrameLimit, int frameSize, IClusterCapacity clusterCapacity) {
+        this.numComputationPartitions = numComputationPartitions;
+        this.frameSize = frameSize;
+        this.groupByMemorySize = groupFrameLimit * (long) frameSize;
+        this.joinMemorySize = joinFrameLimit * (long) frameSize;
+        this.sortMemorySize = sortFrameLimit * (long) frameSize;
+        this.clusterCapacity = clusterCapacity;
+        this.clusterCapacity.setAggregatedCores(1); // At least one core is needed.
+    }
+
+    @Override
+    public Void visitAggregateOperator(AggregateOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitRunningAggregateOperator(RunningAggregateOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitGroupByOperator(GroupByOperator op, Void arg) throws AlgebricksException {
+        calculateMemoryUsageForBlockingOperators(op, groupByMemorySize);
+        return null;
+    }
+
+    @Override
+    public Void visitLimitOperator(LimitOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitInnerJoinOperator(InnerJoinOperator op, Void arg) throws AlgebricksException {
+        calculateMemoryUsageForBlockingOperators(op, joinMemorySize);
+        return null;
+    }
+
+    @Override
+    public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) throws AlgebricksException {
+        calculateMemoryUsageForBlockingOperators(op, joinMemorySize);
+        return null;
+    }
+
+    @Override
+    public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitOrderOperator(OrderOperator op, Void arg) throws AlgebricksException {
+        calculateMemoryUsageForBlockingOperators(op, sortMemorySize);
+        return null;
+    }
+
+    @Override
+    public Void visitAssignOperator(AssignOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitDelegateOperator(DelegateOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitProjectOperator(ProjectOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {
+        // Makes sure that the downstream of a replicate operator is only visited once.
+        if (!visitedOperators.contains(op)) {
+            visitedOperators.add(op);
+            visitInternal(op, true);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitSplitOperator(SplitOperator op, Void arg) throws AlgebricksException {
+        // Makes sure that the downstream of a split operator is only visited once.
+        if (!visitedOperators.contains(op)) {
+            visitedOperators.add(op);
+            visitInternal(op, true);
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitUnionOperator(UnionAllOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitIntersectOperator(IntersectOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitUnnestOperator(UnnestOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitDistinctOperator(DistinctOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitExchangeOperator(ExchangeOperator op, Void arg) throws AlgebricksException {
+        calculateMemoryUsageForExchange(op);
+        return null;
+    }
+
+    @Override
+    public Void visitWriteOperator(WriteOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitInsertDeleteUpsertOperator(InsertDeleteUpsertOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Void arg)
+            throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    @Override
+    public Void visitTokenizeOperator(TokenizeOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
+    // Calculates the memory usage for exchange operators.
+    private void calculateMemoryUsageForExchange(ExchangeOperator op) throws AlgebricksException {
+        visitInternal(op, false);
+        IPhysicalOperator physicalOperator = op.getPhysicalOperator();
+        PhysicalOperatorTag physicalOperatorTag = physicalOperator.getOperatorTag();
+        if (physicalOperatorTag == PhysicalOperatorTag.ONE_TO_ONE_EXCHANGE
+                || physicalOperatorTag == PhysicalOperatorTag.SORT_MERGE_EXCHANGE) {
+            addOutputBuffer(op);
+            return;
+        }
+        stageMemorySoFar += 2L * MAX_BUFFER_PER_CONNECTION * numComputationPartitions * numComputationPartitions
+                * frameSize;
+        clusterCapacity.setAggregatedMemoryByteSize(stageMemorySoFar);
+    }
+
+    // Calculates the cluster-wide memory usage for blocking activities like group-by, sort, and join.
+    private void calculateMemoryUsageForBlockingOperators(ILogicalOperator op, long memSize)
+            throws AlgebricksException {
+        visitInternal(op, false);
+        if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED
+                || op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.LOCAL) {
+            stageMemorySoFar += memSize * numComputationPartitions;
+        } else {
+            stageMemorySoFar += memSize;
+        }
+        clusterCapacity.setAggregatedMemoryByteSize(stageMemorySoFar);
+    }
+
+    // Recursively visits input operators of an operator and sets the CPU core usage.
+    private void visitInternal(ILogicalOperator op, boolean toAddOuputBuffer) throws AlgebricksException {
+        for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
+            inputOpRef.getValue().accept(this, null);
+        }
+        if (toAddOuputBuffer) {
+            addOutputBuffer(op);
+        }
+        setAvailableCores(op);
+    }
+
+    // Adds output buffer for an operator.
+    private void addOutputBuffer(ILogicalOperator op) {
+        if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED
+                || op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.LOCAL) {
+            stageMemorySoFar += frameSize * numComputationPartitions; // every operator needs one output buffer.
+        } else {
+            stageMemorySoFar += frameSize; // every operator needs one output buffer.
+        }
+        clusterCapacity.setAggregatedMemoryByteSize(stageMemorySoFar);
+    }
+
+    // Sets the number of available cores
+    private void setAvailableCores(ILogicalOperator op) {
+        if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED
+                || op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.LOCAL) {
+            clusterCapacity.setAggregatedCores((int) numComputationPartitions);
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index 19c00db..5756e7d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -30,7 +30,6 @@
 
 import org.apache.asterix.active.ActiveLifecycleListener;
 import org.apache.asterix.api.http.servlet.APIServlet;
-import org.apache.asterix.api.http.servlet.FullAPIServlet;
 import org.apache.asterix.api.http.servlet.ClusterAPIServlet;
 import org.apache.asterix.api.http.servlet.ClusterCCDetailsAPIServlet;
 import org.apache.asterix.api.http.servlet.ClusterNodeDetailsAPIServlet;
@@ -38,6 +37,7 @@
 import org.apache.asterix.api.http.servlet.DDLAPIServlet;
 import org.apache.asterix.api.http.servlet.DiagnosticsAPIServlet;
 import org.apache.asterix.api.http.servlet.FeedServlet;
+import org.apache.asterix.api.http.servlet.FullAPIServlet;
 import org.apache.asterix.api.http.servlet.QueryAPIServlet;
 import org.apache.asterix.api.http.servlet.QueryResultAPIServlet;
 import org.apache.asterix.api.http.servlet.QueryServiceServlet;
@@ -47,8 +47,8 @@
 import org.apache.asterix.api.http.servlet.ShutdownAPIServlet;
 import org.apache.asterix.api.http.servlet.UpdateAPIServlet;
 import org.apache.asterix.api.http.servlet.VersionAPIServlet;
-import org.apache.asterix.app.cc.ResourceIdManager;
 import org.apache.asterix.app.cc.CompilerExtensionManager;
+import org.apache.asterix.app.cc.ResourceIdManager;
 import org.apache.asterix.app.external.ExternalLibraryUtils;
 import org.apache.asterix.common.api.AsterixThreadFactory;
 import org.apache.asterix.common.config.AsterixExtension;
@@ -62,11 +62,13 @@
 import org.apache.asterix.metadata.api.IAsterixStateProxy;
 import org.apache.asterix.metadata.bootstrap.AsterixStateProxy;
 import org.apache.asterix.metadata.cluster.ClusterManagerProvider;
+import org.apache.asterix.runtime.job.resource.JobCapacityController;
 import org.apache.asterix.runtime.util.AppContextInfo;
 import org.apache.hyracks.api.application.ICCApplicationContext;
 import org.apache.hyracks.api.application.ICCApplicationEntryPoint;
 import org.apache.hyracks.api.client.HyracksConnection;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
 import org.apache.hyracks.api.lifecycle.LifeCycleComponentManager;
 import org.apache.hyracks.api.messages.IMessageBroker;
 import org.apache.hyracks.control.cc.ClusterControllerService;
@@ -85,6 +87,7 @@
     private static IAsterixStateProxy proxy;
     protected ICCApplicationContext appCtx;
     protected CompilerExtensionManager ccExtensionManager;
+    private IJobCapacityController jobCapacityController;
 
     @Override
     public void start(ICCApplicationContext ccAppCtx, String[] args) throws Exception {
@@ -131,6 +134,8 @@
 
         ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE);
         ccAppCtx.setMessageBroker(messageBroker);
+
+        jobCapacityController = new JobCapacityController(controllerService.getResourceManager());
     }
 
     protected List<AsterixExtension> getExtensions() {
@@ -330,6 +335,11 @@
         ClusterManagerProvider.getClusterManager().notifyStartupCompleted();
     }
 
+    @Override
+    public IJobCapacityController getJobCapacityController() {
+        return jobCapacityController;
+    }
+
     public static synchronized void setAsterixStateProxy(IAsterixStateProxy proxy) {
         CCApplicationEntryPoint.proxy = proxy;
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
index 7a4ff13..41d8b0d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.hyracks.bootstrap;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -93,7 +94,7 @@
     }
 
     @Override
-    public void notifyNodeFailure(Set<String> deadNodeIds) throws HyracksException {
+    public void notifyNodeFailure(Collection<String> deadNodeIds) throws HyracksException {
         for (String deadNode : deadNodeIds) {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("NC: " + deadNode + " left");
@@ -118,7 +119,7 @@
         }
     }
 
-    private void updateProgress(ClusterEventType eventType, Set<String> nodeIds) {
+    private void updateProgress(ClusterEventType eventType, Collection<String> nodeIds) {
         List<IClusterManagementWorkResponse> completedResponses = new ArrayList<IClusterManagementWorkResponse>();
         boolean isComplete = false;
         for (IClusterManagementWorkResponse resp : pendingWorkResponses) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
index f64f998..d437b5b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.hyracks.bootstrap;
 
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
@@ -59,7 +60,7 @@
     }
 
     @Override
-    public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds) {
+    public Set<IClusterManagementWork> notifyNodeFailure(Collection<String> deadNodeIds) {
         setState(ClusterStateManager.INSTANCE.getState());
         ClusterStateManager.INSTANCE.setGlobalRecoveryCompleted(false);
         return Collections.emptySet();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
index 8998c6b..bc270df 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
@@ -32,6 +32,7 @@
 import org.apache.asterix.common.api.IAppRuntimeContext;
 import org.apache.asterix.common.config.AsterixExtension;
 import org.apache.asterix.common.config.MetadataProperties;
+import org.apache.asterix.common.config.StorageProperties;
 import org.apache.asterix.common.config.TransactionProperties;
 import org.apache.asterix.common.config.ClusterProperties;
 import org.apache.asterix.common.config.IPropertiesProvider;
@@ -51,6 +52,7 @@
 import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.api.application.INCApplicationContext;
 import org.apache.hyracks.api.application.INCApplicationEntryPoint;
+import org.apache.hyracks.api.job.resource.NodeCapacity;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
 import org.apache.hyracks.api.lifecycle.LifeCycleComponentManager;
 import org.apache.hyracks.api.messages.IMessageBroker;
@@ -261,6 +263,19 @@
         performLocalCleanUp();
     }
 
+    @Override
+    public NodeCapacity getCapacity() {
+        IPropertiesProvider propertiesProvider = (IPropertiesProvider) runtimeContext;
+        StorageProperties storageProperties = propertiesProvider.getStorageProperties();
+        // Deducts the reserved buffer cache size and memory component size from the maxium heap size,
+        // and deducts one core for processing heartbeats.
+        long memorySize = Runtime.getRuntime().maxMemory() - storageProperties.getBufferCacheSize()
+                - storageProperties.getMemoryComponentGlobalBudget();
+        int allCores = Runtime.getRuntime().availableProcessors();
+        int maximumCoresForComputation = allCores > 1 ? allCores - 1 : allCores;
+        return new NodeCapacity(memorySize, maximumCoresForComputation);
+    }
+
     private void performLocalCleanUp() {
         //Delete working area files from failed jobs
         runtimeContext.getIOManager().deleteWorkspaceFiles();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
index d1d7ff7..d785cce 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.api.util.JavaSerializationUtils;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 
 public class CCMessageBroker implements ICCMessageBroker {
 
@@ -49,8 +50,8 @@
 
     @Override
     public void sendApplicationMessageToNC(IApplicationMessage msg, String nodeId) throws Exception {
-        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
-        NodeControllerState state = nodeMap.get(nodeId);
+        INodeManager nodeManager = ccs.getNodeManager();
+        NodeControllerState state = nodeManager.getNodeControllerState(nodeId);
         state.getNodeController().sendApplicationMessageToNC(JavaSerializationUtils.serialize(msg), null, nodeId);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/ResourceUtils.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/ResourceUtils.java
new file mode 100644
index 0000000..50a21bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/ResourceUtils.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.util;
+
+import org.apache.asterix.app.resource.RequiredCapacityVisitor;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.api.job.resource.ClusterCapacity;
+import org.apache.hyracks.api.job.resource.IClusterCapacity;
+
+public class ResourceUtils {
+
+    private ResourceUtils() {
+    }
+
+    /**
+     * Calculates the required cluster capacity from a given query plan, the computation locations,
+     * the operator memory budgets, and frame size.
+     *
+     * @param plan,
+     *            a given query plan.
+     * @param computationLocations,
+     *            the partitions for computation.
+     * @param sortFrameLimit,
+     *            the frame limit for one sorter partition.
+     * @param groupFrameLimit,
+     *            the frame limit for one group-by partition.
+     * @param joinFrameLimit
+     *            the frame limit for one joiner partition.
+     * @param frameSize
+     *            the frame size used in query execution.
+     * @return the required cluster capacity for executing the query.
+     * @throws AlgebricksException
+     *             if the query plan is malformed.
+     */
+    public static IClusterCapacity getRequiredCompacity(ILogicalPlan plan,
+            AlgebricksAbsolutePartitionConstraint computationLocations, int sortFrameLimit, int groupFrameLimit,
+            int joinFrameLimit, int frameSize)
+            throws AlgebricksException {
+        // Creates a cluster capacity visitor.
+        IClusterCapacity clusterCapacity = new ClusterCapacity();
+        RequiredCapacityVisitor visitor = new RequiredCapacityVisitor(computationLocations.getLocations().length,
+                sortFrameLimit, groupFrameLimit, joinFrameLimit, frameSize, clusterCapacity);
+
+        // There could be only one root operator for a top-level query plan.
+        ILogicalOperator rootOp = plan.getRoots().get(0).getValue();
+        rootOp.accept(visitor, null);
+        return clusterCapacity;
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/APIFrameworkTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/APIFrameworkTest.java
index a79053e..e041021 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/APIFrameworkTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/APIFrameworkTest.java
@@ -27,8 +27,10 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.client.IClusterInfoCollector;
 import org.apache.hyracks.api.client.NodeControllerInfo;
 import org.junit.Assert;
@@ -39,15 +41,16 @@
 public class APIFrameworkTest {
 
     @Test
-    public void testGetComputationLocations() throws Exception {
+    public void testChooseLocations() throws Exception {
+        // Mocks cluster info collector.
         IClusterInfoCollector clusterInfoCollector = mock(IClusterInfoCollector.class);
 
         // Constructs mocked cluster nodes.
         Map<String, NodeControllerInfo> map = new HashMap<>();
         NodeControllerInfo nc1Info = mock(NodeControllerInfo.class);
-        when(nc1Info.getNumCores()).thenReturn(4);
+        when(nc1Info.getNumAvailableCores()).thenReturn(1);
         NodeControllerInfo nc2Info = mock(NodeControllerInfo.class);
-        when(nc2Info.getNumCores()).thenReturn(4);
+        when(nc2Info.getNumAvailableCores()).thenReturn(1);
         String nc1 = "nc1";
         String nc2 = "nc2";
         map.put(nc1, nc1Info);
@@ -57,10 +60,56 @@
         // Creates an APIFramework.
         APIFramework apiFramework = new APIFramework(mock(ILangCompilationProvider.class));
 
+        // Tests large storage locations.
+        AlgebricksAbsolutePartitionConstraint storageLocations = new AlgebricksAbsolutePartitionConstraint(
+                new String[] { "node1", "node1", "node2" });
+        AlgebricksAbsolutePartitionConstraint computationLocations = (AlgebricksAbsolutePartitionConstraint) PA
+                .invokeMethod(apiFramework,
+                        "chooseLocations(" + IClusterInfoCollector.class.getName() + ",int,"
+                                + AlgebricksAbsolutePartitionConstraint.class.getName() + ")",
+                        clusterInfoCollector, CompilerProperties.COMPILER_PARALLELISM_AS_STORAGE, storageLocations);
+        Assert.assertTrue(computationLocations.getLocations().length == 2);
+
+        // Tests suitable storage locations.
+        storageLocations = new AlgebricksAbsolutePartitionConstraint(new String[] { "node1", "node2" });
+        computationLocations = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(apiFramework,
+                "chooseLocations(" + IClusterInfoCollector.class.getName() + ",int,"
+                        + AlgebricksAbsolutePartitionConstraint.class.getName() + ")",
+                clusterInfoCollector, CompilerProperties.COMPILER_PARALLELISM_AS_STORAGE, storageLocations);
+        Assert.assertTrue(computationLocations.getLocations().length == 2);
+
+        // Tests small storage locations.
+        storageLocations = new AlgebricksAbsolutePartitionConstraint(new String[] { "node1" });
+        computationLocations = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(apiFramework,
+                "chooseLocations(" + IClusterInfoCollector.class.getName() + ",int,"
+                        + AlgebricksAbsolutePartitionConstraint.class.getName() + ")",
+                clusterInfoCollector, CompilerProperties.COMPILER_PARALLELISM_AS_STORAGE, storageLocations);
+        Assert.assertTrue(computationLocations.getLocations().length == 1);
+
+        // Verifies the number of calls on clusterInfoCollector.getNodeControllerInfos() in
+        // APIFramework.chooseLocations(...).
+        verify(clusterInfoCollector, times(3)).getNodeControllerInfos();
+    }
+
+    @Test
+    public void testGetComputationLocations() throws AlgebricksException {
+        // Constructs mocked cluster nodes.
+        Map<String, NodeControllerInfo> map = new HashMap<>();
+        NodeControllerInfo nc1Info = mock(NodeControllerInfo.class);
+        when(nc1Info.getNumAvailableCores()).thenReturn(4);
+        NodeControllerInfo nc2Info = mock(NodeControllerInfo.class);
+        when(nc2Info.getNumAvailableCores()).thenReturn(4);
+        String nc1 = "nc1";
+        String nc2 = "nc2";
+        map.put(nc1, nc1Info);
+        map.put(nc2, nc2Info);
+
+        // Creates an APIFramework.
+        APIFramework apiFramework = new APIFramework(mock(ILangCompilationProvider.class));
+
         // Tests odd number parallelism.
         AlgebricksAbsolutePartitionConstraint loc = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(
-                apiFramework, "getComputationLocations(" + IClusterInfoCollector.class.getName() + ",int)",
-                clusterInfoCollector, 5);
+                apiFramework, "getComputationLocations(java.util.Map,int)", map, 5);
         int nc1Count = 0, nc2Count = 0;
         String[] partitions = loc.getLocations();
         for (String partition : partitions) {
@@ -78,7 +127,7 @@
 
         // Tests even number parallelism.
         loc = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(apiFramework,
-                "getComputationLocations(" + IClusterInfoCollector.class.getName() + ",int)", clusterInfoCollector, 8);
+                "getComputationLocations(java.util.Map,int)", map, 8);
         nc1Count = 0;
         nc2Count = 0;
         partitions = loc.getLocations();
@@ -93,40 +142,35 @@
         Assert.assertTrue(nc1Count > 0);
         Assert.assertTrue(nc2Count > 0);
         Assert.assertTrue(Math.abs(nc1Count - nc2Count) == 0); // Tests load balance.
-        // The maximum parallelism cannot be beyond n *(#core-1), where n is the number of NCs and #core is the number
+        // The maximum parallelism cannot be beyond n * core, where n is the number of NCs and #core is the number
         // of cores per NC.
-        Assert.assertTrue(partitions.length == 6);
+        Assert.assertTrue(partitions.length == 8);
 
         // Tests the case when parallelism is one.
         loc = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(apiFramework,
-                "getComputationLocations(" + IClusterInfoCollector.class.getName() + ",int)", clusterInfoCollector, 1);
+                "getComputationLocations(java.util.Map,int)", map, 1);
         Assert.assertTrue(loc.getLocations().length == 1);
 
         // Tests the case when parallelism is a negative.
         // In this case, the compiler has no idea and falls back to the default setting where all possible cores
         // are used.
         loc = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(apiFramework,
-                "getComputationLocations(" + IClusterInfoCollector.class.getName() + ",int)", clusterInfoCollector,
-                -100);
-        Assert.assertTrue(loc.getLocations().length == 6);
+                "getComputationLocations(java.util.Map,int)", map, -100);
+        Assert.assertTrue(loc.getLocations().length == 8);
 
         // Tests the case when parallelism is -1.
         // In this case, the compiler has no idea and falls back to the default setting where all possible cores
         // are used.
         loc = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(apiFramework,
-                "getComputationLocations(" + IClusterInfoCollector.class.getName() + ",int)", clusterInfoCollector, -1);
-        Assert.assertTrue(loc.getLocations().length == 6);
+                "getComputationLocations(java.util.Map,int)", map, -1);
+        Assert.assertTrue(loc.getLocations().length == 8);
 
         // Tests the case when parallelism is zero.
         // In this case, the compiler has no idea and falls back to the default setting where all possible cores
         // are used.
         loc = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(apiFramework,
-                "getComputationLocations(" + IClusterInfoCollector.class.getName() + ",int)", clusterInfoCollector, 0);
-        Assert.assertTrue(loc.getLocations().length == 6);
-
-        // Verifies the number of calls on clusterInfoCollector.getNodeControllerInfos() in
-        // APIFramework.getComputationLocations(...).
-        verify(clusterInfoCollector, times(6)).getNodeControllerInfos();
+                "getComputationLocations(java.util.Map,int)", map, 0);
+        Assert.assertTrue(loc.getLocations().length == 8);
     }
 
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/RequiredCapacityVisitorTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/RequiredCapacityVisitorTest.java
new file mode 100644
index 0000000..cc18c31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/RequiredCapacityVisitorTest.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.app.resource;
+
+import java.util.Collections;
+
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
+import org.apache.hyracks.api.job.resource.ClusterCapacity;
+import org.apache.hyracks.api.job.resource.IClusterCapacity;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class RequiredCapacityVisitorTest {
+
+    private static final long MEMORY_BUDGET = 33554432L;
+    private static final int FRAME_SIZE = 32768;
+    private static final int FRAME_LIMIT = (int) (MEMORY_BUDGET / FRAME_SIZE);
+    private static final int PARALLELISM = 10;
+
+    @Test
+    public void testParallelGroupBy() throws AlgebricksException {
+        IClusterCapacity clusterCapacity = new ClusterCapacity();
+        RequiredCapacityVisitor visitor = makeComputationCapacityVisitor(PARALLELISM, clusterCapacity);
+
+        // Constructs a parallel group-by query plan.
+        GroupByOperator globalGby = makeGroupByOperator(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+        ExchangeOperator exchange = new ExchangeOperator();
+        exchange.setPhysicalOperator(new HashPartitionExchangePOperator(Collections.emptyList(), null));
+        GroupByOperator localGby = makeGroupByOperator(AbstractLogicalOperator.ExecutionMode.LOCAL);
+        globalGby.getInputs().add(new MutableObject<>(exchange));
+        exchange.getInputs().add(new MutableObject<>(localGby));
+
+        // Verifies the calculated cluster capacity requirement for the test quer plan.
+        globalGby.accept(visitor, null);
+        Assert.assertTrue(clusterCapacity.getAggregatedCores() == PARALLELISM);
+        Assert.assertTrue(clusterCapacity.getAggregatedMemoryByteSize() == 2 * MEMORY_BUDGET * PARALLELISM
+                + 2 * FRAME_SIZE * PARALLELISM * PARALLELISM);
+    }
+
+    @Test
+    public void testUnPartitionedGroupBy() throws AlgebricksException {
+        IClusterCapacity clusterCapacity = new ClusterCapacity();
+        RequiredCapacityVisitor visitor = makeComputationCapacityVisitor(PARALLELISM, clusterCapacity);
+
+        // Constructs a parallel group-by query plan.
+        GroupByOperator globalGby = makeGroupByOperator(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+        ExchangeOperator exchange = new ExchangeOperator();
+        exchange.setPhysicalOperator(new OneToOneExchangePOperator());
+        exchange.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+        GroupByOperator localGby = makeGroupByOperator(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+        globalGby.getInputs().add(new MutableObject<>(exchange));
+        exchange.getInputs().add(new MutableObject<>(localGby));
+
+        // Verifies the calculated cluster capacity requirement for the test quer plan.
+        globalGby.accept(visitor, null);
+        Assert.assertTrue(clusterCapacity.getAggregatedCores() == 1);
+        Assert.assertTrue(clusterCapacity.getAggregatedMemoryByteSize() == 2 * MEMORY_BUDGET + FRAME_SIZE);
+    }
+
+    @Test
+    public void testParallelJoin() throws AlgebricksException {
+        IClusterCapacity clusterCapacity = new ClusterCapacity();
+        RequiredCapacityVisitor visitor = makeComputationCapacityVisitor(PARALLELISM, clusterCapacity);
+
+        // Constructs a join query plan.
+        InnerJoinOperator join = makeJoinOperator(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+
+        // Left child plan of the join.
+        ExchangeOperator leftChildExchange = new ExchangeOperator();
+        leftChildExchange.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+        leftChildExchange.setPhysicalOperator(new HashPartitionExchangePOperator(Collections.emptyList(), null));
+        InnerJoinOperator leftChild = makeJoinOperator(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+        join.getInputs().add(new MutableObject<>(leftChildExchange));
+        leftChildExchange.getInputs().add(new MutableObject<>(leftChild));
+        EmptyTupleSourceOperator ets = new EmptyTupleSourceOperator();
+        ets.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+        leftChild.getInputs().add(new MutableObject<>(ets));
+        leftChild.getInputs().add(new MutableObject<>(ets));
+
+        // Right child plan of the join.
+        ExchangeOperator rightChildExchange = new ExchangeOperator();
+        rightChildExchange.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+        rightChildExchange.setPhysicalOperator(new HashPartitionExchangePOperator(Collections.emptyList(), null));
+        GroupByOperator rightChild = makeGroupByOperator(AbstractLogicalOperator.ExecutionMode.LOCAL);
+        join.getInputs().add(new MutableObject<>(rightChildExchange));
+        rightChildExchange.getInputs().add(new MutableObject<>(rightChild));
+        rightChild.getInputs().add(new MutableObject<>(ets));
+
+        // Verifies the calculated cluster capacity requirement for the test quer plan.
+        join.accept(visitor, null);
+        Assert.assertTrue(clusterCapacity.getAggregatedCores() == PARALLELISM);
+        Assert.assertTrue(clusterCapacity.getAggregatedMemoryByteSize() == 3 * MEMORY_BUDGET * PARALLELISM
+                + 2 * 2L * PARALLELISM * PARALLELISM * FRAME_SIZE + 3 * FRAME_SIZE * PARALLELISM);
+    }
+
+    @Test
+    public void testUnPartitionedJoin() throws AlgebricksException {
+        IClusterCapacity clusterCapacity = new ClusterCapacity();
+        RequiredCapacityVisitor visitor = makeComputationCapacityVisitor(PARALLELISM, clusterCapacity);
+
+        // Constructs a join query plan.
+        InnerJoinOperator join = makeJoinOperator(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+
+        // Left child plan of the join.
+        ExchangeOperator leftChildExchange = new ExchangeOperator();
+        leftChildExchange.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+        leftChildExchange.setPhysicalOperator(new OneToOneExchangePOperator());
+        InnerJoinOperator leftChild = makeJoinOperator(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+        join.getInputs().add(new MutableObject<>(leftChildExchange));
+        leftChildExchange.getInputs().add(new MutableObject<>(leftChild));
+        EmptyTupleSourceOperator ets = new EmptyTupleSourceOperator();
+        ets.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+        leftChild.getInputs().add(new MutableObject<>(ets));
+        leftChild.getInputs().add(new MutableObject<>(ets));
+
+        // Right child plan of the join.
+        ExchangeOperator rightChildExchange = new ExchangeOperator();
+        rightChildExchange.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+        rightChildExchange.setPhysicalOperator(new OneToOneExchangePOperator());
+        GroupByOperator rightChild = makeGroupByOperator(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+        join.getInputs().add(new MutableObject<>(rightChildExchange));
+        rightChildExchange.getInputs().add(new MutableObject<>(rightChild));
+        rightChild.getInputs().add(new MutableObject<>(ets));
+
+        // Verifies the calculated cluster capacity requirement for the test quer plan.
+        join.accept(visitor, null);
+        Assert.assertTrue(clusterCapacity.getAggregatedCores() == 1);
+        Assert.assertTrue(clusterCapacity.getAggregatedMemoryByteSize() == 3 * MEMORY_BUDGET + 5L * FRAME_SIZE);
+    }
+
+    private RequiredCapacityVisitor makeComputationCapacityVisitor(int numComputationPartitions,
+            IClusterCapacity clusterCapacity) {
+        return new RequiredCapacityVisitor(numComputationPartitions, FRAME_LIMIT, FRAME_LIMIT, FRAME_LIMIT, FRAME_SIZE,
+                clusterCapacity);
+    }
+
+    private GroupByOperator makeGroupByOperator(AbstractLogicalOperator.ExecutionMode exeMode) {
+        GroupByOperator groupByOperator = new GroupByOperator();
+        groupByOperator.setExecutionMode(exeMode);
+        return groupByOperator;
+    }
+
+    private InnerJoinOperator makeJoinOperator(AbstractLogicalOperator.ExecutionMode exeMode) {
+        InnerJoinOperator joinOperator = new InnerJoinOperator(new MutableObject<>(ConstantExpression.TRUE));
+        joinOperator.setExecutionMode(exeMode);
+        return joinOperator;
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q09/q09.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q09/q09.3.query.sqlpp
index 774747e..e0ae61c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q09/q09.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q09/q09.3.query.sqlpp
@@ -20,6 +20,9 @@
 
 use tpcds;
 
+set `compiler.joinmemory` "4MB"
+set `compiler.groupmemory` "4MB"
+
 select case when (select value count(ss)
                   from store_sales ss
                   where ss_quantity >= 1 and ss_quantity <= 20) > 25437
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterEventsSubscriber.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterEventsSubscriber.java
index 3a98b82..fef4e31 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterEventsSubscriber.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterEventsSubscriber.java
@@ -18,6 +18,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+import java.util.Collection;
 import java.util.Set;
 
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
@@ -28,7 +29,7 @@
      * @param deadNodeIds
      * @return
      */
-    public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds);
+    public Set<IClusterManagementWork> notifyNodeFailure(Collection<String> deadNodeIds);
 
     /**
      * @param joinedNodeId
diff --git a/asterixdb/asterix-installer/src/test/resources/integrationts/asterix-configuration.xml b/asterixdb/asterix-installer/src/test/resources/integrationts/asterix-configuration.xml
index 9992009..a5ecc6b 100644
--- a/asterixdb/asterix-installer/src/test/resources/integrationts/asterix-configuration.xml
+++ b/asterixdb/asterix-installer/src/test/resources/integrationts/asterix-configuration.xml
@@ -20,7 +20,7 @@
 
   <property>
     <name>nc.java.opts</name>
-    <value>-Xmx3096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"</value>
+    <value>-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"</value>
     <description>JVM parameters for each Node Contoller (NC)</description>
   </property>
 
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/RemoveNodeWorkResponse.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/RemoveNodeWorkResponse.java
index 580aab7..34b873c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/RemoveNodeWorkResponse.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/RemoveNodeWorkResponse.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.metadata.cluster;
 
+import java.util.Collection;
 import java.util.HashSet;
 import java.util.Set;
 
@@ -30,7 +31,7 @@
         nodesToBeRemoved.addAll(w.getNodesToBeRemoved());
     }
 
-    public boolean updateProgress(Set<String> failedNodeIds) {
+    public boolean updateProgress(Collection<String> failedNodeIds) {
         nodesToBeRemoved.removeAll(failedNodeIds);
         return nodesToBeRemoved.isEmpty();
 
diff --git a/asterixdb/asterix-runtime/pom.xml b/asterixdb/asterix-runtime/pom.xml
index 1ccdc76..6458fb0 100644
--- a/asterixdb/asterix-runtime/pom.xml
+++ b/asterixdb/asterix-runtime/pom.xml
@@ -140,5 +140,16 @@
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
     </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>1.10.19</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/resource/JobCapacityController.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/resource/JobCapacityController.java
new file mode 100644
index 0000000..8ea1fa7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/job/resource/JobCapacityController.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.job.resource;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.resource.IClusterCapacity;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
+import org.apache.hyracks.api.job.resource.IReadOnlyClusterCapacity;
+import org.apache.hyracks.control.cc.scheduler.IResourceManager;
+
+// To avoid the computation cost for checking the capacity constraint for each node,
+// currently the admit/allocation decisions are based on the aggregated resource information.
+// TODO(buyingyi): investigate partition-aware resource control.
+public class JobCapacityController implements IJobCapacityController {
+
+    private final IResourceManager resourceManager;
+
+    public JobCapacityController(IResourceManager resourceManager) {
+        this.resourceManager = resourceManager;
+    }
+
+    @Override
+    public JobSubmissionStatus allocate(JobSpecification job) throws HyracksException {
+        IClusterCapacity requiredCapacity = job.getRequiredClusterCapacity();
+        long reqAggregatedMemoryByteSize = requiredCapacity.getAggregatedMemoryByteSize();
+        int reqAggregatedNumCores = requiredCapacity.getAggregatedCores();
+        IReadOnlyClusterCapacity maximumCapacity = resourceManager.getMaximumCapacity();
+        if (!(reqAggregatedMemoryByteSize <= maximumCapacity.getAggregatedMemoryByteSize()
+                && reqAggregatedNumCores <= maximumCapacity.getAggregatedCores())) {
+            throw HyracksException.create(ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY, requiredCapacity.toString(),
+                    maximumCapacity.toString());
+        }
+        IClusterCapacity currentCapacity = resourceManager.getCurrentCapacity();
+        long currentAggregatedMemoryByteSize = currentCapacity.getAggregatedMemoryByteSize();
+        int currentAggregatedAvailableCores = currentCapacity.getAggregatedCores();
+        if (!(reqAggregatedMemoryByteSize <= currentAggregatedMemoryByteSize
+                && reqAggregatedNumCores <= currentAggregatedAvailableCores)) {
+            return JobSubmissionStatus.QUEUE;
+        }
+        currentCapacity.setAggregatedMemoryByteSize(currentAggregatedMemoryByteSize - reqAggregatedMemoryByteSize);
+        currentCapacity.setAggregatedCores(currentAggregatedAvailableCores - reqAggregatedNumCores);
+        return JobSubmissionStatus.EXECUTE;
+    }
+
+    @Override
+    public void release(JobSpecification job) {
+        IClusterCapacity requiredCapacity = job.getRequiredClusterCapacity();
+        long reqAggregatedMemoryByteSize = requiredCapacity.getAggregatedMemoryByteSize();
+        int reqAggregatedNumCores = requiredCapacity.getAggregatedCores();
+        IClusterCapacity currentCapacity = resourceManager.getCurrentCapacity();
+        long aggregatedMemoryByteSize = currentCapacity.getAggregatedMemoryByteSize();
+        int aggregatedNumCores = currentCapacity.getAggregatedCores();
+        currentCapacity.setAggregatedMemoryByteSize(aggregatedMemoryByteSize + reqAggregatedMemoryByteSize);
+        currentCapacity.setAggregatedCores(aggregatedNumCores + reqAggregatedNumCores);
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/RuntimeUtils.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/RuntimeUtils.java
index ed93a2c..608def7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/RuntimeUtils.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/util/RuntimeUtils.java
@@ -28,6 +28,7 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 
 /**
  * Utility class for obtaining information on the set of Hyracks NodeController
@@ -61,6 +62,7 @@
     public static void getNodeControllerMap(Map<InetAddress, Set<String>> map) {
         ClusterControllerService ccs = (ClusterControllerService) AppContextInfo.INSTANCE
                 .getCCApplicationContext().getControllerService();
-        map.putAll(ccs.getIpAddressNodeNameMap());
+        INodeManager nodeManager = ccs.getNodeManager();
+        map.putAll(nodeManager.getIpAddressNodeNameMap());
     }
 }
diff --git a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/job/resource/JobCapacityControllerTest.java b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/job/resource/JobCapacityControllerTest.java
new file mode 100644
index 0000000..4a63885
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/job/resource/JobCapacityControllerTest.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.job.resource;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.resource.ClusterCapacity;
+import org.apache.hyracks.api.job.resource.IClusterCapacity;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
+import org.apache.hyracks.api.job.resource.NodeCapacity;
+import org.apache.hyracks.control.cc.scheduler.IResourceManager;
+import org.apache.hyracks.control.cc.scheduler.ResourceManager;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class JobCapacityControllerTest {
+
+    @Test
+    public void test() throws HyracksException {
+        IResourceManager resourceManager = makeResourceManagerWithCapacity(4294967296L, 33);
+        JobCapacityController capacityController = new JobCapacityController(resourceManager);
+
+        // Verifies the correctness of the allocate method.
+        Assert.assertTrue(capacityController.allocate(
+                makeJobWithRequiredCapacity(4294967296L, 16)) == IJobCapacityController.JobSubmissionStatus.EXECUTE);
+        Assert.assertTrue(capacityController.allocate(
+                makeJobWithRequiredCapacity(2147483648L, 16)) == IJobCapacityController.JobSubmissionStatus.QUEUE);
+        Assert.assertTrue(capacityController.allocate(
+                makeJobWithRequiredCapacity(2147483648L, 32)) == IJobCapacityController.JobSubmissionStatus.QUEUE);
+
+        boolean exceedCapacity = false;
+        try {
+            capacityController.allocate(makeJobWithRequiredCapacity(2147483648L, 64));
+        } catch (HyracksException e) {
+            exceedCapacity = e.getErrorCode() == ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY;
+        }
+        Assert.assertTrue(exceedCapacity);
+        Assert.assertTrue(capacityController.allocate(
+                makeJobWithRequiredCapacity(4294967296L, 32)) == IJobCapacityController.JobSubmissionStatus.QUEUE);
+        exceedCapacity = false;
+        try {
+            capacityController.allocate(makeJobWithRequiredCapacity(4294967297L, 33));
+        } catch (HyracksException e) {
+            exceedCapacity = e.getErrorCode() == ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY;
+        }
+        Assert.assertTrue(exceedCapacity);
+
+        // Verifies that the release method does not leak resource.
+        capacityController.release(makeJobWithRequiredCapacity(4294967296L, 16));
+        Assert.assertTrue(resourceManager.getCurrentCapacity().getAggregatedMemoryByteSize() == 4294967296L);
+        Assert.assertTrue(resourceManager.getCurrentCapacity().getAggregatedCores() == 33);
+    }
+
+    private IResourceManager makeResourceManagerWithCapacity(long memorySize, int cores) throws HyracksException {
+        IResourceManager resourceManager = new ResourceManager();
+        resourceManager.update("node1", new NodeCapacity(memorySize, cores));
+        return resourceManager;
+    }
+
+    private JobSpecification makeJobWithRequiredCapacity(long memorySize, int cores) {
+        // Generates cluster capacity.
+        IClusterCapacity clusterCapacity = makeComputationCapacity(memorySize, cores);
+
+        // Generates a job.
+        JobSpecification job = mock(JobSpecification.class);
+        when(job.getRequiredClusterCapacity()).thenReturn(clusterCapacity);
+        return job;
+    }
+
+    private IClusterCapacity makeComputationCapacity(long memorySize, int cores) {
+        IClusterCapacity clusterCapacity = new ClusterCapacity();
+        clusterCapacity.setAggregatedMemoryByteSize(memorySize);
+        clusterCapacity.setAggregatedCores(cores);
+        return clusterCapacity;
+    }
+
+}
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/SampleLocalClusterIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/SampleLocalClusterIT.java
index 4a5f90f..83d421f 100644
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/SampleLocalClusterIT.java
+++ b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/SampleLocalClusterIT.java
@@ -78,9 +78,11 @@
 
     @Test
     public void test0_startCluster() throws Exception {
-        Process process = new ProcessBuilder(joinPath(LOCAL_SAMPLES_DIR, "bin/start-sample-cluster.sh"))
+        Process process = new ProcessBuilder(joinPath(LOCAL_SAMPLES_DIR, "bin/stop-sample-cluster.sh"), "-f")
                 .inheritIO().start();
         Assert.assertEquals(0, process.waitFor());
+        process = new ProcessBuilder(joinPath(LOCAL_SAMPLES_DIR, "bin/start-sample-cluster.sh")).inheritIO().start();
+        Assert.assertEquals(0, process.waitFor());
     }
 
     @Test
diff --git a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
index 92bc076..774c4d9 100644
--- a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
+++ b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
@@ -46,22 +46,42 @@
         return new AlgebricksException(ErrorCode.HYRACKS, errorCode, ErrorCode.getErrorMessage(errorCode), params);
     }
 
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
     public AlgebricksException(String message) {
         this(ErrorMessageUtil.NONE, UNKNOWN, message, null, null);
     }
 
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
     public AlgebricksException(Throwable cause) {
         this(ErrorMessageUtil.NONE, UNKNOWN, cause.getMessage(), cause, null);
     }
 
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
     public AlgebricksException(Throwable cause, String nodeId) {
         this(ErrorMessageUtil.NONE, UNKNOWN, cause.getMessage(), cause, nodeId);
     }
 
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
     public AlgebricksException(String message, Throwable cause, String nodeId) {
         this(ErrorMessageUtil.NONE, UNKNOWN, message, cause, nodeId);
     }
 
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
     public AlgebricksException(String message, Throwable cause) {
         this(ErrorMessageUtil.NONE, UNKNOWN, message, cause, (String) null);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleController.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleController.java
index 9446756..7328278 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleController.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleController.java
@@ -59,7 +59,7 @@
         if (ruleCollection instanceof List) {
             rules = (List<IAlgebraicRewriteRule>) ruleCollection;
         } else {
-            throw AlgebricksException.create(ErrorCode.COMPILATION_RULECOLLECTION_NOT_INSTANCE_OF_LIST,
+            throw AlgebricksException.create(ErrorCode.RULECOLLECTION_NOT_INSTANCE_OF_LIST,
                     this.getClass().getName());
         }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-api/pom.xml b/hyracks-fullstack/hyracks/hyracks-api/pom.xml
index 7f132dd..2bb6cc2 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-api/pom.xml
@@ -93,5 +93,10 @@
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
     </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCApplicationContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCApplicationContext.java
index 8b83d83..5f4877d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCApplicationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCApplicationContext.java
@@ -22,6 +22,7 @@
 
 import org.apache.hyracks.api.context.ICCContext;
 import org.apache.hyracks.api.job.IJobLifecycleListener;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
 
 /**
  * Application Context at the Cluster Controller for an application.
@@ -38,7 +39,7 @@
      * @param state
      *            The distributed state
      */
-    public void setDistributedState(Serializable state);
+    void setDistributedState(Serializable state);
 
     /**
      * A listener that listens to Job Lifecycle events at the Cluster
@@ -46,21 +47,21 @@
      *
      * @param jobLifecycleListener
      */
-    public void addJobLifecycleListener(IJobLifecycleListener jobLifecycleListener);
+    void addJobLifecycleListener(IJobLifecycleListener jobLifecycleListener);
 
     /**
      * A listener that listens to Cluster Lifecycle events at the Cluster
      * Controller.
      *
-     * @param jobLifecycleListener
+     * @param clusterLifecycleListener
      */
-    public void addClusterLifecycleListener(IClusterLifecycleListener clusterLifecycleListener);
+    void addClusterLifecycleListener(IClusterLifecycleListener clusterLifecycleListener);
 
     /**
      * Get the Cluster Controller Context.
      *
      * @return The Cluster Controller Context.
      */
-    public ICCContext getCCContext();
+    ICCContext getCCContext();
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCApplicationEntryPoint.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCApplicationEntryPoint.java
index 9f7f222..c11cc7a 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCApplicationEntryPoint.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/ICCApplicationEntryPoint.java
@@ -18,10 +18,14 @@
  */
 package org.apache.hyracks.api.application;
 
-public interface ICCApplicationEntryPoint {
-    public void start(ICCApplicationContext ccAppCtx, String[] args) throws Exception;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
 
-    public void stop() throws Exception;
+public interface ICCApplicationEntryPoint {
+    void start(ICCApplicationContext ccAppCtx, String[] args) throws Exception;
+
+    void stop() throws Exception;
 
     void startupCompleted() throws Exception;
+
+    IJobCapacityController getJobCapacityController();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IClusterLifecycleListener.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IClusterLifecycleListener.java
index a9bef18..191a4af 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IClusterLifecycleListener.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/IClusterLifecycleListener.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hyracks.api.application;
 
+import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
 
@@ -47,6 +48,6 @@
      * @param deadNodeIds
      *            A set of Node Controller Ids that have left the cluster. The set is not cumulative.
      */
-    public void notifyNodeFailure(Set<String> deadNodeIds) throws HyracksException;
+    public void notifyNodeFailure(Collection<String> deadNodeIds) throws HyracksException;
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/INCApplicationEntryPoint.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/INCApplicationEntryPoint.java
index ea850c5..dea6e4b 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/INCApplicationEntryPoint.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/application/INCApplicationEntryPoint.java
@@ -18,10 +18,14 @@
  */
 package org.apache.hyracks.api.application;
 
+import org.apache.hyracks.api.job.resource.NodeCapacity;
+
 public interface INCApplicationEntryPoint {
-    public void start(INCApplicationContext ncAppCtx, String[] args) throws Exception;
+    void start(INCApplicationContext ncAppCtx, String[] args) throws Exception;
 
-    public void notifyStartupComplete() throws Exception;
+    void notifyStartupComplete() throws Exception;
 
-    public void stop() throws Exception;
+    void stop() throws Exception;
+
+    NodeCapacity getCapacity();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java
index c90644f..57f389f 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java
@@ -35,16 +35,16 @@
 
     private final NetworkAddress messagingNetworkAddress;
 
-    private final int numCores;
+    private final int numAvailableCores;
 
     public NodeControllerInfo(String nodeId, NodeStatus status, NetworkAddress netAddress,
-            NetworkAddress datasetNetworkAddress, NetworkAddress messagingNetworkAddress, int numCores) {
+            NetworkAddress datasetNetworkAddress, NetworkAddress messagingNetworkAddress, int numAvailableCores) {
         this.nodeId = nodeId;
         this.status = status;
         this.netAddress = netAddress;
         this.datasetNetworkAddress = datasetNetworkAddress;
         this.messagingNetworkAddress = messagingNetworkAddress;
-        this.numCores = numCores;
+        this.numAvailableCores = numAvailableCores;
     }
 
     public String getNodeId() {
@@ -67,7 +67,7 @@
         return messagingNetworkAddress;
     }
 
-    public int getNumCores() {
-        return numCores;
+    public int getNumAvailableCores() {
+        return numAvailableCores;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
index 00c2cc4..9a908ab 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
@@ -75,10 +75,8 @@
         acg.setFrameSize(spec.getFrameSize());
         acg.setMaxReattempts(spec.getMaxReattempts());
         acg.setJobletEventListenerFactory(spec.getJobletEventListenerFactory());
-        acg.setGlobalJobDataFactory(spec.getGlobalJobDataFactory());
         acg.setConnectorPolicyAssignmentPolicy(spec.getConnectorPolicyAssignmentPolicy());
         acg.setUseConnectorPolicyForScheduling(spec.isUseConnectorPolicyForScheduling());
-        acg.setReportTaskDetails(spec.isReportTaskDetails());
         final Set<Constraint> constraints = new HashSet<Constraint>();
         final IConstraintAcceptor acceptor = new IConstraintAcceptor() {
             @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index 5a67188..1ca8bb3 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -27,19 +27,28 @@
 /**
  * A registry of runtime/compile error codes
  * Error code:
- * 0 --- 999: runtime errors
- * 1000 ---- 1999: compilation errors
+ * 0 --- 9999: runtime errors
+ * 10000 ---- 19999: compilation errors
  */
 public class ErrorCode {
     private static final String RESOURCE_PATH = "errormsg" + File.separator + "en.properties";
     public static final String HYRACKS = "HYR";
 
+    // Runtime error codes.
     public static final int INVALID_OPERATOR_OPERATION = 1;
     public static final int ERROR_PROCESSING_TUPLE = 2;
     public static final int FAILURE_ON_NODE = 3;
-    public static final int RUNTIME_FILE_WITH_ABSOULTE_PATH_NOT_WITHIN_ANY_IO_DEVICE = 4;
-    public static final int RUNTIME_FULLTEXT_PHRASE_FOUND = 5;
-    public static final int COMPILATION_RULECOLLECTION_NOT_INSTANCE_OF_LIST = 1001;
+    public static final int FILE_WITH_ABSOULTE_PATH_NOT_WITHIN_ANY_IO_DEVICE = 4;
+    public static final int FULLTEXT_PHRASE_FOUND = 5;
+    public static final int JOB_QUEUE_FULL = 6;
+    public static final int INVALID_NETWORK_ADDRESS = 7;
+    public static final int INVALID_INPUT_PARAMETER = 8;
+    public static final int JOB_REQUIREMENTS_EXCEED_CAPACITY = 9;
+    public static final int NO_SUCH_NODE = 10;
+    public static final int CLASS_LOADING_ISSUE = 11;
+
+    // Compilation error codes.
+    public static final int RULECOLLECTION_NOT_INSTANCE_OF_LIST = 10001;
 
     // Loads the map that maps error codes to error message templates.
     private static Map<Integer, String> errorMessageMap = null;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
index 56be93e..0fd6923 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
@@ -16,10 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.api.exceptions;
 
 import java.io.Serializable;
-import java.util.logging.Logger;
 
 import org.apache.hyracks.api.util.ErrorMessageUtil;
 
@@ -27,14 +27,6 @@
  * The main execution time exception type for runtime errors in a hyracks environment
  */
 public class HyracksDataException extends HyracksException {
-    private static final long serialVersionUID = 1L;
-
-    public static final int UNKNOWN = 0;
-    private final String component;
-    private final int errorCode;
-    private final Serializable[] params;
-    private final String nodeId;
-    private transient volatile String msgCache;
 
     public static HyracksDataException create(int code, Serializable... params) {
         return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), params);
@@ -46,76 +38,68 @@
 
     public HyracksDataException(String component, int errorCode, String message, Throwable cause, String nodeId,
             Serializable... params) {
-        super(message, cause);
-        this.component = component;
-        this.errorCode = errorCode;
-        this.nodeId = nodeId;
-        this.params = params;
+        super(component, errorCode, message, cause, nodeId, params);
     }
 
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
     public HyracksDataException(String message) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, message, null, null);
+        super(message);
     }
 
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
     public HyracksDataException(Throwable cause) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, cause.getMessage(), cause, null);
+        super(cause);
     }
 
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
     public HyracksDataException(Throwable cause, String nodeId) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, cause.getMessage(), cause, nodeId);
+        super(cause, nodeId);
     }
 
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
     public HyracksDataException(String message, Throwable cause, String nodeId) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, message, cause, nodeId);
+        super(message, cause, nodeId);
     }
 
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
     public HyracksDataException(String message, Throwable cause) {
-        this(ErrorMessageUtil.NONE, UNKNOWN, message, cause, (String) null);
+        super(message, cause);
     }
 
     public HyracksDataException(String component, int errorCode, Serializable... params) {
-        this(component, errorCode, null, null, null, params);
+        super(component, errorCode, null, null, null, params);
     }
 
     public HyracksDataException(Throwable cause, int errorCode, Serializable... params) {
-        this(ErrorMessageUtil.NONE, errorCode, cause.getMessage(), cause, null, params);
+        super(ErrorMessageUtil.NONE, errorCode, cause.getMessage(), cause, null, params);
     }
 
     public HyracksDataException(String component, int errorCode, String message, Serializable... params) {
-        this(component, errorCode, message, null, null, params);
+        super(component, errorCode, message, null, null, params);
     }
 
     public HyracksDataException(String component, int errorCode, Throwable cause, Serializable... params) {
-        this(component, errorCode, cause.getMessage(), cause, null, params);
+        super(component, errorCode, cause.getMessage(), cause, null, params);
     }
 
     public HyracksDataException(String component, int errorCode, String message, Throwable cause,
             Serializable... params) {
-        this(component, errorCode, message, cause, null, params);
-    }
-
-    public String getComponent() {
-        return component;
-    }
-
-    public int getErrorCode() {
-        return errorCode;
-    }
-
-    public Object[] getParams() {
-        return params;
-    }
-
-    public String getNodeId() {
-        return nodeId;
-    }
-
-    @Override
-    public String getMessage() {
-        if (msgCache == null) {
-            msgCache = ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), params);
-        }
-        return msgCache;
+        super(component, errorCode, message, cause, null, params);
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
index e939d26..5d13212 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
@@ -19,22 +19,118 @@
 package org.apache.hyracks.api.exceptions;
 
 import java.io.IOException;
+import java.io.Serializable;
+
+import org.apache.hyracks.api.util.ErrorMessageUtil;
 
 public class HyracksException extends IOException {
     private static final long serialVersionUID = 1L;
 
-    public HyracksException() {
+    public static final int UNKNOWN = 0;
+    private final String component;
+    private final int errorCode;
+    private final Serializable[] params;
+    private final String nodeId;
+    private transient volatile String msgCache;
+
+    public static HyracksException create(int code, Serializable... params) {
+        return new HyracksException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), params);
     }
 
-    public HyracksException(String message) {
-        super(message);
+    public static HyracksException create(int code, Throwable cause, Serializable... params) {
+        return new HyracksException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), cause, params);
     }
 
-    public HyracksException(Throwable cause) {
-        super(cause);
-    }
-
-    public HyracksException(String message, Throwable cause) {
+    public HyracksException(String component, int errorCode, String message, Throwable cause, String nodeId,
+            Serializable... params) {
         super(message, cause);
+        this.component = component;
+        this.errorCode = errorCode;
+        this.nodeId = nodeId;
+        this.params = params;
+    }
+
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
+    public HyracksException(String message) {
+        this(ErrorMessageUtil.NONE, UNKNOWN, message, null, null);
+    }
+
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
+    public HyracksException(Throwable cause) {
+        this(ErrorMessageUtil.NONE, UNKNOWN, cause.getMessage(), cause, null);
+    }
+
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
+    public HyracksException(Throwable cause, String nodeId) {
+        this(ErrorMessageUtil.NONE, UNKNOWN, cause.getMessage(), cause, nodeId);
+    }
+
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
+    public HyracksException(String message, Throwable cause, String nodeId) {
+        this(ErrorMessageUtil.NONE, UNKNOWN, message, cause, nodeId);
+    }
+
+    /**
+     * @deprecated Error code is needed.
+     */
+    @Deprecated
+    public HyracksException(String message, Throwable cause) {
+        this(ErrorMessageUtil.NONE, UNKNOWN, message, cause, (String) null);
+    }
+
+    public HyracksException(String component, int errorCode, Serializable... params) {
+        this(component, errorCode, null, null, null, params);
+    }
+
+    public HyracksException(Throwable cause, int errorCode, Serializable... params) {
+        this(ErrorMessageUtil.NONE, errorCode, cause.getMessage(), cause, null, params);
+    }
+
+    public HyracksException(String component, int errorCode, String message, Serializable... params) {
+        this(component, errorCode, message, null, null, params);
+    }
+
+    public HyracksException(String component, int errorCode, Throwable cause, Serializable... params) {
+        this(component, errorCode, cause.getMessage(), cause, null, params);
+    }
+
+    public HyracksException(String component, int errorCode, String message, Throwable cause, Serializable... params) {
+        this(component, errorCode, message, cause, null, params);
+    }
+
+    public String getComponent() {
+        return component;
+    }
+
+    public int getErrorCode() {
+        return errorCode;
+    }
+
+    public Object[] getParams() {
+        return params;
+    }
+
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    @Override
+    public String getMessage() {
+        if (msgCache == null) {
+            msgCache = ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), params);
+        }
+        return msgCache;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
index 84a961e..5787c72 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobSpecification.java
@@ -28,10 +28,6 @@
 import java.util.Map;
 import java.util.Set;
 
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.api.constraints.Constraint;
 import org.apache.hyracks.api.constraints.expressions.ConstantExpression;
@@ -44,6 +40,12 @@
 import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.dataset.ResultSetId;
+import org.apache.hyracks.api.job.resource.ClusterCapacity;
+import org.apache.hyracks.api.job.resource.IClusterCapacity;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class JobSpecification implements Serializable, IOperatorDescriptorRegistry, IConnectorDescriptorRegistry {
     private static final long serialVersionUID = 1L;
@@ -76,11 +78,9 @@
 
     private IJobletEventListenerFactory jobletEventListenerFactory;
 
-    private IGlobalJobDataFactory globalJobDataFactory;
-
     private boolean useConnectorPolicyForScheduling;
 
-    private boolean reportTaskDetails;
+    private IClusterCapacity requiredClusterCapacity;
 
     private transient int operatorIdCounter;
 
@@ -106,7 +106,7 @@
         connectorIdCounter = 0;
         maxReattempts = 2;
         useConnectorPolicyForScheduling = false;
-        reportTaskDetails = true;
+        requiredClusterCapacity = new ClusterCapacity();
         setFrameSize(frameSize);
     }
 
@@ -281,14 +281,6 @@
         this.jobletEventListenerFactory = jobletEventListenerFactory;
     }
 
-    public IGlobalJobDataFactory getGlobalJobDataFactory() {
-        return globalJobDataFactory;
-    }
-
-    public void setGlobalJobDataFactory(IGlobalJobDataFactory globalJobDataFactory) {
-        this.globalJobDataFactory = globalJobDataFactory;
-    }
-
     public boolean isUseConnectorPolicyForScheduling() {
         return useConnectorPolicyForScheduling;
     }
@@ -297,12 +289,12 @@
         this.useConnectorPolicyForScheduling = useConnectorPolicyForScheduling;
     }
 
-    public boolean isReportTaskDetails() {
-        return reportTaskDetails;
+    public void setRequiredClusterCapacity(IClusterCapacity capacity) {
+        this.requiredClusterCapacity = capacity;
     }
 
-    public void setReportTaskDetails(boolean reportTaskDetails) {
-        this.reportTaskDetails = reportTaskDetails;
+    public IClusterCapacity getRequiredClusterCapacity() {
+        return requiredClusterCapacity;
     }
 
     private <K, V> void insertIntoIndexedMap(Map<K, List<V>> map, K key, int index, V value) {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobStatus.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobStatus.java
index 4351e39..50db00d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobStatus.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/JobStatus.java
@@ -16,11 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.api.job;
 
 public enum JobStatus {
-    INITIALIZED,
+    PENDING,
     RUNNING,
     TERMINATED,
     FAILURE,
+    FAILURE_BEFORE_EXECUTION
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/ClusterCapacity.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/ClusterCapacity.java
new file mode 100644
index 0000000..ded4b63
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/ClusterCapacity.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.api.job.resource;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.lang3.ObjectUtils;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksException;
+
+public class ClusterCapacity implements IClusterCapacity {
+
+    private long aggregatedMemoryByteSize = 0;
+    private int aggregatedCores = 0;
+    private final Map<String, Long> nodeMemoryMap = new HashMap<>();
+    private final Map<String, Integer> nodeCoreMap = new HashMap<>();
+
+    @Override
+    public long getAggregatedMemoryByteSize() {
+        return aggregatedMemoryByteSize;
+    }
+
+    @Override
+    public int getAggregatedCores() {
+        return aggregatedCores;
+    }
+
+    @Override
+    public long getMemoryByteSize(String nodeId) throws HyracksException {
+        if (!nodeMemoryMap.containsKey(nodeId)) {
+            throw HyracksException.create(ErrorCode.NO_SUCH_NODE, nodeId);
+        }
+        return nodeMemoryMap.get(nodeId);
+    }
+
+    @Override
+    public int getCores(String nodeId) throws HyracksException {
+        if (!nodeMemoryMap.containsKey(nodeId)) {
+            throw HyracksException.create(ErrorCode.NO_SUCH_NODE, nodeId);
+        }
+        return nodeCoreMap.get(nodeId);
+    }
+
+    @Override
+    public void setAggregatedMemoryByteSize(long aggregatedMemoryByteSize) {
+        this.aggregatedMemoryByteSize = aggregatedMemoryByteSize;
+    }
+
+    @Override
+    public void setAggregatedCores(int aggregatedCores) {
+        this.aggregatedCores = aggregatedCores;
+    }
+
+    @Override
+    public void setMemoryByteSize(String nodeId, long memoryByteSize) {
+        nodeMemoryMap.put(nodeId, memoryByteSize);
+    }
+
+    @Override
+    public void setCores(String nodeId, int cores) {
+        nodeCoreMap.put(nodeId, cores);
+    }
+
+    @Override
+    public void update(String nodeId, NodeCapacity nodeCapacity) throws HyracksException {
+        // Removes the existing node resource and the aggregated resource statistics.
+        if (nodeMemoryMap.containsKey(nodeId)) {
+            aggregatedMemoryByteSize -= nodeMemoryMap.remove(nodeId);
+        }
+        if (nodeCoreMap.containsKey(nodeId)) {
+            aggregatedCores -= nodeCoreMap.remove(nodeId);
+        }
+
+        long memorySize = nodeCapacity.getMemoryByteSize();
+        int cores = nodeCapacity.getCores();
+        // Updates the node capacity map when both memory size and cores are positive.
+        if (memorySize > 0 && cores > 0) {
+            aggregatedMemoryByteSize += memorySize;
+            aggregatedCores += cores;
+            nodeMemoryMap.put(nodeId, memorySize);
+            nodeCoreMap.put(nodeId, cores);
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return ObjectUtils.hashCodeMulti(aggregatedMemoryByteSize, aggregatedCores, nodeMemoryMap,
+                nodeCoreMap);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof ClusterCapacity)) {
+            return false;
+        }
+        ClusterCapacity capacity = (ClusterCapacity) o;
+        return aggregatedMemoryByteSize == capacity.aggregatedMemoryByteSize
+                && aggregatedCores == capacity.aggregatedCores
+                && ObjectUtils.equals(nodeMemoryMap, capacity.nodeMemoryMap)
+                && ObjectUtils.equals(nodeCoreMap, capacity.nodeCoreMap);
+    }
+
+    @Override
+    public String toString() {
+        return "capacity (memory: " + aggregatedMemoryByteSize + " bytes, CPU cores: " + aggregatedCores + ")";
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/DefaultJobCapacityController.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/DefaultJobCapacityController.java
new file mode 100644
index 0000000..9e38a20
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/DefaultJobCapacityController.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.api.job.resource;
+
+import org.apache.hyracks.api.job.JobSpecification;
+
+public class DefaultJobCapacityController implements IJobCapacityController {
+
+    public static final DefaultJobCapacityController INSTANCE = new DefaultJobCapacityController();
+
+    private DefaultJobCapacityController() {
+    }
+
+    @Override
+    public JobSubmissionStatus allocate(JobSpecification job) {
+        return JobSubmissionStatus.EXECUTE;
+    }
+
+    @Override
+    public void release(JobSpecification job) {
+        // No operation here.
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IClusterCapacity.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IClusterCapacity.java
new file mode 100644
index 0000000..ac3261d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IClusterCapacity.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.api.job.resource;
+
+import org.apache.hyracks.api.exceptions.HyracksException;
+
+/**
+ * This interface abstracts the mutable capacity for a cluster.
+ */
+public interface IClusterCapacity extends IReadOnlyClusterCapacity {
+
+    /**
+     * Sets the aggregated memory size for a cluster.
+     *
+     * @param aggregatedMemoryByteSize,
+     *            the aggregated memory size.
+     */
+    void setAggregatedMemoryByteSize(long aggregatedMemoryByteSize);
+
+    /**
+     * Sets the aggregated number of CPU cores for a cluster.
+     *
+     * @param aggregatedCores,
+     *            the total number of cores.
+     */
+    void setAggregatedCores(int aggregatedCores);
+
+    /**
+     * Sets the memory byte size (for computation) of a specific node.
+     *
+     * @param nodeId,
+     *            the node id.
+     * @param memoryByteSize,
+     *            the available memory byte size for computation of the node.
+     */
+    void setMemoryByteSize(String nodeId, long memoryByteSize);
+
+    /**
+     * Sets the number of CPU cores for a specific node.
+     *
+     * @param nodeId,
+     *            the node id.
+     * @param cores,
+     *            the number of CPU cores for the node.
+     */
+    void setCores(String nodeId, int cores);
+
+    /**
+     * Updates the cluster capacity information with the capacity of one particular node.
+     *
+     * @param nodeId,
+     *            the id of the node for updating.
+     * @param capacity,
+     *            the capacity of one particular node.
+     * @throws HyracksException
+     *             when the parameters are invalid.
+     */
+    void update(String nodeId, NodeCapacity capacity) throws HyracksException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IJobCapacityController.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IJobCapacityController.java
new file mode 100644
index 0000000..5fa4bd9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IJobCapacityController.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.api.job.resource;
+
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.JobSpecification;
+
+/**
+ * This interface determines the behavior of a job when it is submitted to the job manager.
+ * The job could be one of the following three cases:
+ * -- rejected immediately because its capacity requirement exceeds the cluster's capacity.
+ * -- entered into a pending job queue for deferred execution, due to the current capacity limitation because of
+ * concurrent running jobs;
+ * -- executed immediately because there is sufficient capacity.
+ */
+public interface IJobCapacityController {
+
+    enum JobSubmissionStatus {
+        EXECUTE,
+        QUEUE
+    }
+
+    /**
+     * Allocates required cluster capacity for a job.
+     *
+     * @param job,
+     *            the job specification.
+     * @return EXECUTE, if the job can be executed immediately;
+     *         QUEUE, if the job cannot be executed
+     * @throws HyracksException
+     *             if the job's capacity requirement exceeds the maximum capacity of the cluster.
+     */
+    JobSubmissionStatus allocate(JobSpecification job) throws HyracksException;
+
+    /**
+     * Releases cluster capacity for a job when it completes.
+     *
+     * @param job,
+     *            the job specification.
+     */
+    void release(JobSpecification job);
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IReadOnlyClusterCapacity.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IReadOnlyClusterCapacity.java
new file mode 100644
index 0000000..59b6bfd
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/IReadOnlyClusterCapacity.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.api.job.resource;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.exceptions.HyracksException;
+
+/**
+ * This interface provides read-only methods for the capacity of a cluster.
+ */
+public interface IReadOnlyClusterCapacity extends Serializable {
+
+    /**
+     * @return the aggregated memory byte size for the cluster.
+     */
+    long getAggregatedMemoryByteSize();
+
+    /**
+     * @return the aggregated number of cores
+     */
+    int getAggregatedCores();
+
+    /**
+     * Retrieves the memory byte size for computation on a specific node.
+     * (Note that usually a portion of memory is used for storage.)
+     *
+     * @param nodeId,
+     *            the node id.
+     * @return the memory byte size for computation on the node.
+     * @throws HyracksException
+     *             when the input node does not exist.
+     */
+    long getMemoryByteSize(String nodeId) throws HyracksException;
+
+    /**
+     * Retrieves the number of CPU cores for computation on a specific node.
+     *
+     * @param nodeId,
+     *            the node id.
+     * @return the number of CPU cores for computation on the node.
+     * @throws HyracksException,
+     *             when the input node does not exist.
+     */
+    int getCores(String nodeId) throws HyracksException;
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/NodeCapacity.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/NodeCapacity.java
new file mode 100644
index 0000000..7902e7d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/job/resource/NodeCapacity.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.api.job.resource;
+
+import java.io.Serializable;
+
+/**
+ * Specifies the capacity for computation on a particular node, i.e., a NCDriver process.
+ */
+public class NodeCapacity implements Serializable {
+
+    // All memory for computations -- this is not changed during the lifetime of a running instance.
+    private final long memoryByteSize;
+
+    // All CPU cores -- currently we assume that it doesn't change during the lifetime of a running instance.
+    // Otherwise, for each heartbeat, we will have to update global cluster capacity of a cluster.
+    private final int cores;
+
+    /**
+     * NOTE: neither parameters can be negative.
+     * However, both of them can be zero, which means the node is to be removed from the cluster.
+     *
+     * @param memorySize,
+     *            the memory size of the node.
+     * @param cores,
+     *            the number of cores of the node.
+     */
+    public NodeCapacity(long memorySize, int cores) {
+        this.memoryByteSize = memorySize;
+        this.cores = cores;
+    }
+
+    public long getMemoryByteSize() {
+        return memoryByteSize;
+    }
+
+    public int getCores() {
+        return cores;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
index 52367ee..d17c9aa 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -17,9 +17,18 @@
 # under the License.
 #
 
-1 = Unsupported operation %1$s in %2$s operator
-2 = Error in processing tuple %1$s in a frame
-4 = The file with absolute path %1$s is not within any of the current IO devices
-5 = Phrase search in Full-text is not supported. An expression should include only one word
+# 0 --- 9999: runtime errors
+# 10000 ---- 19999: compilation errors
 
-1001 = The given rule collection %1$s is not an instance of the List class.
\ No newline at end of file
+1=Unsupported operation %1$s in %2$s operator
+2=Error in processing tuple %1$s in a frame
+4=The file with absolute path %1$s is not within any of the current IO devices
+5=Phrase search in Full-text is not supported. An expression should include only one word
+6=Job queue is full with %1$s jobs
+7=Network address cannot be resolved -- %1$s
+8=Invalid internal input parameter
+9=Job requirement %1$s exceeds capacity %2$s
+10=Node %1$s does not exist
+11=Class loading issue: %1$s
+
+10000 = The given rule collection %1$s is not an instance of the List class.
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/job/resource/ClusterCapacityTest.java b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/job/resource/ClusterCapacityTest.java
new file mode 100644
index 0000000..277e8e2
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/job/resource/ClusterCapacityTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.api.job.resource;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ClusterCapacityTest {
+
+    @Test
+    public void test() throws HyracksException {
+        ClusterCapacity capacity = new ClusterCapacity();
+        String nodeId = "node1";
+
+        // Adds one node.
+        capacity.update(nodeId, new NodeCapacity(1024L, 8));
+        Assert.assertTrue(capacity.getAggregatedMemoryByteSize() == 1024L);
+        Assert.assertTrue(capacity.getAggregatedCores() == 8);
+
+        // Updates the node.
+        capacity.update(nodeId, new NodeCapacity(-1L, -2));
+
+        // Verifies that node is removed
+        Assert.assertTrue(capacity.getAggregatedMemoryByteSize() == 0L);
+        Assert.assertTrue(capacity.getAggregatedCores() == 0);
+
+        boolean nodeNotExist = false;
+        try {
+            capacity.getMemoryByteSize(nodeId);
+        } catch (HyracksException e) {
+            nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+        }
+        Assert.assertTrue(nodeNotExist);
+        nodeNotExist = false;
+        try {
+            capacity.getCores(nodeId);
+        } catch (HyracksException e) {
+            nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+        }
+        Assert.assertTrue(nodeNotExist);
+
+        // Adds the node again.
+        capacity.update(nodeId, new NodeCapacity(1024L, 8));
+        // Updates the node.
+        capacity.update(nodeId, new NodeCapacity(4L, 0));
+
+        // Verifies that node does not exist
+        Assert.assertTrue(capacity.getAggregatedMemoryByteSize() == 0L);
+        Assert.assertTrue(capacity.getAggregatedCores() == 0);
+        nodeNotExist = false;
+        try {
+            capacity.getMemoryByteSize(nodeId);
+        } catch (HyracksException e) {
+            nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+        }
+        Assert.assertTrue(nodeNotExist);
+        nodeNotExist = false;
+        try {
+            capacity.getCores(nodeId);
+        } catch (HyracksException e) {
+            nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+        }
+        Assert.assertTrue(nodeNotExist);
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
index e949e57..105d47f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
@@ -113,5 +113,16 @@
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
     </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>2.0.2-beta</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
index 26beb63..7ea5f70 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
@@ -72,13 +72,13 @@
             case GET_JOB_STATUS:
                 HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf =
                         (HyracksClientInterfaceFunctions.GetJobStatusFunction) fn;
-                ccs.getWorkQueue().schedule(new GetJobStatusWork(ccs, gjsf.getJobId(),
-                        new IPCResponder<JobStatus>(handle, mid)));
+                ccs.getWorkQueue().schedule(
+                        new GetJobStatusWork(ccs.getJobManager(), gjsf.getJobId(), new IPCResponder<>(handle, mid)));
                 break;
             case GET_JOB_INFO:
                 HyracksClientInterfaceFunctions.GetJobInfoFunction gjif =
                         (HyracksClientInterfaceFunctions.GetJobInfoFunction) fn;
-                ccs.getWorkQueue().schedule(new GetJobInfoWork(ccs, gjif.getJobId(),
+                ccs.getWorkQueue().schedule(new GetJobInfoWork(ccs.getJobManager(), gjif.getJobId(),
                         new IPCResponder<JobInfo>(handle, mid)));
                 break;
             case START_JOB:
@@ -118,8 +118,8 @@
                         new IPCResponder<>(handle, mid)));
                 break;
             case GET_NODE_CONTROLLERS_INFO:
-                ccs.getWorkQueue().schedule(new GetNodeControllersInfoWork(ccs,
-                        new IPCResponder<>(handle, mid)));
+                ccs.getWorkQueue().schedule(
+                        new GetNodeControllersInfoWork(ccs.getNodeManager(), new IPCResponder<>(handle, mid)));
                 break;
             case GET_CLUSTER_TOPOLOGY:
                 try {
@@ -149,7 +149,8 @@
             case GET_NODE_DETAILS_JSON:
                 HyracksClientInterfaceFunctions.GetNodeDetailsJSONFunction gndjf =
                         (HyracksClientInterfaceFunctions.GetNodeDetailsJSONFunction) fn;
-                ccs.getWorkQueue().schedule(new GetNodeDetailsJSONWork(ccs, gndjf.getNodeId(),
+                ccs.getWorkQueue()
+                        .schedule(new GetNodeDetailsJSONWork(ccs.getNodeManager(), ccs.getCCConfig(), gndjf.getNodeId(),
                         gndjf.isIncludeStats(), gndjf.isIncludeConfig(), new IPCResponder<>(handle, mid)));
                 break;
             case THREAD_DUMP:
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerIPCI.java
index b6c9a08..21fcf92 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerIPCI.java
@@ -68,7 +68,7 @@
                 break;
             case UNREGISTER_NODE:
                 CCNCFunctions.UnregisterNodeFunction unf = (CCNCFunctions.UnregisterNodeFunction) fn;
-                ccs.getWorkQueue().schedule(new UnregisterNodeWork(ccs, unf.getNodeId()));
+                ccs.getWorkQueue().schedule(new UnregisterNodeWork(ccs.getNodeManager(), unf.getNodeId()));
                 break;
             case NODE_HEARTBEAT:
                 CCNCFunctions.NodeHeartbeatFunction nhf = (CCNCFunctions.NodeHeartbeatFunction) fn;
@@ -87,7 +87,7 @@
                 break;
             case REPORT_PROFILE:
                 CCNCFunctions.ReportProfileFunction rpf = (CCNCFunctions.ReportProfileFunction) fn;
-                ccs.getWorkQueue().schedule(new ReportProfilesWork(ccs, rpf.getProfiles()));
+                ccs.getWorkQueue().schedule(new ReportProfilesWork(ccs.getJobManager(), rpf.getProfiles()));
                 break;
             case NOTIFY_TASK_COMPLETE:
                 CCNCFunctions.NotifyTaskCompleteFunction ntcf = (CCNCFunctions.NotifyTaskCompleteFunction) fn;
@@ -137,7 +137,7 @@
                         rsf.getDeploymentId(), rsf.getNodeId()));
                 break;
             case GET_NODE_CONTROLLERS_INFO:
-                ccs.getWorkQueue().schedule(new GetNodeControllersInfoWork(ccs,
+                ccs.getWorkQueue().schedule(new GetNodeControllersInfoWork(ccs.getNodeManager(),
                         new IResultCallback<Map<String, NodeControllerInfo>>() {
                             @Override
                             public void setValue(Map<String, NodeControllerInfo> result) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
index 5fdcede..1a363c7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClusterControllerService.java
@@ -20,12 +20,12 @@
 
 import java.io.File;
 import java.io.FileReader;
+import java.lang.reflect.Constructor;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -43,17 +43,22 @@
 import org.apache.hyracks.api.context.ICCContext;
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.resource.DefaultJobCapacityController;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
 import org.apache.hyracks.api.service.IControllerService;
 import org.apache.hyracks.api.topology.ClusterTopology;
 import org.apache.hyracks.api.topology.TopologyDefinitionParser;
 import org.apache.hyracks.control.cc.application.CCApplicationContext;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
+import org.apache.hyracks.control.cc.cluster.NodeManager;
 import org.apache.hyracks.control.cc.dataset.DatasetDirectoryService;
 import org.apache.hyracks.control.cc.dataset.IDatasetDirectoryService;
-import org.apache.hyracks.control.cc.job.JobRun;
+import org.apache.hyracks.control.cc.job.IJobManager;
+import org.apache.hyracks.control.cc.scheduler.IResourceManager;
+import org.apache.hyracks.control.cc.scheduler.ResourceManager;
 import org.apache.hyracks.control.cc.web.WebServer;
-import org.apache.hyracks.control.cc.work.GatherStateDumpsWork.StateDumpRun;
 import org.apache.hyracks.control.cc.work.GetIpAddressNodeNameMapWork;
+import org.apache.hyracks.control.cc.work.GatherStateDumpsWork.StateDumpRun;
 import org.apache.hyracks.control.cc.work.GetThreadDumpWork.ThreadDumpRun;
 import org.apache.hyracks.control.cc.work.RemoveDeadNodesWork;
 import org.apache.hyracks.control.cc.work.ShutdownNCServiceWork;
@@ -83,10 +88,6 @@
 
     private final LogFile jobLog;
 
-    private final Map<String, NodeControllerState> nodeRegistry;
-
-    private final Map<InetAddress, Set<String>> ipAddressNodeNameMap;
-
     private final ServerContext serverCtx;
 
     private final WebServer webServer;
@@ -95,12 +96,6 @@
 
     private CCApplicationContext appCtx;
 
-    private final Map<JobId, JobRun> activeRunMap;
-
-    private final Map<JobId, JobRun> runMapArchive;
-
-    private final Map<JobId, List<Exception>> runMapHistory;
-
     private final WorkQueue workQueue;
 
     private ExecutorService executor;
@@ -119,6 +114,12 @@
 
     private final Map<String, ThreadDumpRun> threadDumpRunMap;
 
+    private final INodeManager nodeManager;
+
+    private final IResourceManager resourceManager = new ResourceManager();
+
+    private IJobManager jobManager;
+
     private ShutdownRun shutdownCallback;
 
     private ICCApplicationEntryPoint aep;
@@ -127,8 +128,6 @@
         this.ccConfig = ccConfig;
         File jobLogFolder = new File(ccConfig.ccRoot, "logs/jobs");
         jobLog = new LogFile(jobLogFolder);
-        nodeRegistry = new LinkedHashMap<>();
-        ipAddressNodeNameMap = new HashMap<>();
         serverCtx = new ServerContext(ServerContext.ServerType.CLUSTER_CONTROLLER, new File(ccConfig.ccRoot));
         IIPCI ccIPCI = new ClusterControllerIPCI(this);
         clusterIPC = new IPCSystem(new InetSocketAddress(ccConfig.clusterNetPort), ccIPCI,
@@ -137,25 +136,7 @@
         clientIPC = new IPCSystem(new InetSocketAddress(ccConfig.clientNetIpAddress, ccConfig.clientNetPort), ciIPCI,
                 new JavaSerializationBasedPayloadSerializerDeserializer());
         webServer = new WebServer(this);
-        activeRunMap = new HashMap<>();
-        runMapArchive = new LinkedHashMap<JobId, JobRun>() {
-            private static final long serialVersionUID = 1L;
 
-            @Override
-            protected boolean removeEldestEntry(Map.Entry<JobId, JobRun> eldest) {
-                return size() > ccConfig.jobHistorySize;
-            }
-        };
-        runMapHistory = new LinkedHashMap<JobId, List<Exception>>() {
-            private static final long serialVersionUID = 1L;
-            /** history size + 1 is for the case when history size = 0 */
-            private int allowedSize = 100 * (ccConfig.jobHistorySize + 1);
-
-            @Override
-            protected boolean removeEldestEntry(Map.Entry<JobId, List<Exception>> eldest) {
-                return size() > allowedSize;
-            }
-        };
         // WorkQueue is in charge of heartbeat as well as other events.
         workQueue = new WorkQueue("ClusterController", Thread.MAX_PRIORITY);
         this.timer = new Timer(true);
@@ -167,6 +148,9 @@
         deploymentRunMap = new HashMap<>();
         stateDumpRunMap = new HashMap<>();
         threadDumpRunMap = Collections.synchronizedMap(new HashMap<>());
+
+        // Node manager is in charge of cluster membership management.
+        nodeManager = new NodeManager(ccConfig, resourceManager);
     }
 
     private static ClusterTopology computeClusterTopology(CCConfig ccConfig) throws Exception {
@@ -207,13 +191,22 @@
         appCtx.addJobLifecycleListener(datasetDirectoryService);
         executor = Executors.newCachedThreadPool(appCtx.getThreadFactory());
         String className = ccConfig.appCCMainClass;
+
+        IJobCapacityController jobCapacityController = DefaultJobCapacityController.INSTANCE;
         if (className != null) {
             Class<?> c = Class.forName(className);
             aep = (ICCApplicationEntryPoint) c.newInstance();
             String[] args = ccConfig.appArgs == null ? null
                     : ccConfig.appArgs.toArray(new String[ccConfig.appArgs.size()]);
             aep.start(appCtx, args);
+            jobCapacityController = aep.getJobCapacityController();
         }
+
+        // Job manager is in charge of job lifecycle management.
+        Constructor<?> jobManagerConstructor = this.getClass().getClassLoader().loadClass(ccConfig.jobManagerClassName)
+                .getConstructor(CCConfig.class, ClusterControllerService.class, IJobCapacityController.class);
+        jobManager = (IJobManager) jobManagerConstructor.newInstance(ccConfig, this, jobCapacityController);
+
     }
 
     private void connectNCs() throws Exception {
@@ -301,20 +294,16 @@
         return ccContext;
     }
 
-    public Map<JobId, JobRun> getActiveRunMap() {
-        return activeRunMap;
+    public IJobManager getJobManager() {
+        return jobManager;
     }
 
-    public Map<JobId, JobRun> getRunMapArchive() {
-        return runMapArchive;
+    public INodeManager getNodeManager() {
+        return nodeManager;
     }
 
-    public Map<JobId, List<Exception>> getRunHistory() {
-        return runMapHistory;
-    }
-
-    public Map<InetAddress, Set<String>> getIpAddressNodeNameMap() {
-        return ipAddressNodeNameMap;
+    public IResourceManager getResourceManager() {
+        return resourceManager;
     }
 
     public LogFile getJobLogFile() {
@@ -329,10 +318,6 @@
         return executor;
     }
 
-    public Map<String, NodeControllerState> getNodeMap() {
-        return nodeRegistry;
-    }
-
     public CCConfig getConfig() {
         return ccConfig;
     }
@@ -366,7 +351,8 @@
 
         @Override
         public void getIPAddressNodeMap(Map<InetAddress, Set<String>> map) throws HyracksDataException {
-            GetIpAddressNodeNameMapWork ginmw = new GetIpAddressNodeNameMapWork(ClusterControllerService.this, map);
+            GetIpAddressNodeNameMapWork ginmw = new GetIpAddressNodeNameMapWork(
+                    ClusterControllerService.this.getNodeManager(), map);
             try {
                 workQueue.scheduleAndSync(ginmw);
             } catch (Exception e) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java
index bf94dff..955b7f2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java
@@ -25,10 +25,9 @@
 import java.util.Map;
 import java.util.Set;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.resource.NodeCapacity;
 import org.apache.hyracks.control.common.base.INodeController;
 import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.control.common.controllers.NodeRegistration;
@@ -36,6 +35,9 @@
 import org.apache.hyracks.control.common.heartbeat.HeartbeatSchema;
 import org.apache.hyracks.control.common.heartbeat.HeartbeatSchema.GarbageCollectorInfo;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class NodeControllerState {
     private static final int RRD_SIZE = 720;
 
@@ -141,7 +143,7 @@
 
     private int lastHeartbeatDuration;
 
-    private int numCores;
+    private NodeCapacity capacity;
 
     public NodeControllerState(INodeController nodeController, NodeRegistration reg) {
         this.nodeController = nodeController;
@@ -204,7 +206,7 @@
         diskWrites = new long[RRD_SIZE];
 
         rrdPtr = 0;
-        numCores = 0;
+        capacity = reg.getCapacity();
     }
 
     public synchronized void notifyHeartbeat(HeartbeatData hbData) {
@@ -242,7 +244,6 @@
             diskReads[rrdPtr] = hbData.diskReads;
             diskWrites[rrdPtr] = hbData.diskWrites;
             rrdPtr = (rrdPtr + 1) % RRD_SIZE;
-            numCores = hbData.numCores;
         }
     }
 
@@ -250,10 +251,6 @@
         return lastHeartbeatDuration++;
     }
 
-    public int getLastHeartbeatDuration() {
-        return lastHeartbeatDuration;
-    }
-
     public INodeController getNodeController() {
         return nodeController;
     }
@@ -277,8 +274,9 @@
     public NetworkAddress getMessagingPort() {
         return messagingPort;
     }
-    public int getNumCores() {
-        return numCores;
+
+    public NodeCapacity getCapacity() {
+        return capacity;
     }
 
     public synchronized ObjectNode toSummaryJSON()  {
@@ -324,6 +322,8 @@
             o.putPOJO("nonheap-used-sizes", nonheapUsedSize);
             o.putPOJO("nonheap-committed-sizes", nonheapCommittedSize);
             o.putPOJO("nonheap-max-sizes", nonheapMaxSize);
+            o.putPOJO("application-memory-budget", capacity.getMemoryByteSize());
+            o.putPOJO("application-cpu-core-budget", capacity.getCores());
             o.putPOJO("thread-counts", threadCount);
             o.putPOJO("peak-thread-counts", peakThreadCount);
             o.putPOJO("system-load-averages", systemLoadAverage);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/adminconsole/pages/IndexPage.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/adminconsole/pages/IndexPage.java
index 680c2a7..443fa88 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/adminconsole/pages/IndexPage.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/adminconsole/pages/IndexPage.java
@@ -42,7 +42,7 @@
     public IndexPage() throws Exception {
         ClusterControllerService ccs = getAdminConsoleApplication().getClusterControllerService();
 
-        GetNodeSummariesJSONWork gnse = new GetNodeSummariesJSONWork(ccs);
+        GetNodeSummariesJSONWork gnse = new GetNodeSummariesJSONWork(ccs.getNodeManager());
         ccs.getWorkQueue().scheduleAndSync(gnse);
         ArrayNode nodeSummaries = gnse.getSummaries();
         add(new Label("node-count", String.valueOf(nodeSummaries.size())));
@@ -63,7 +63,7 @@
         };
         add(nodeList);
 
-        GetJobSummariesJSONWork gjse = new GetJobSummariesJSONWork(ccs);
+        GetJobSummariesJSONWork gjse = new GetJobSummariesJSONWork(ccs.getJobManager());
         ccs.getWorkQueue().scheduleAndSync(gjse);
         ArrayNode jobSummaries = gjse.getSummaries();
         ListView<JsonNode> jobList = new ListView<JsonNode>("jobs-list", Lists.newArrayList(jobSummaries.iterator())) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java
index abc07d9..68add85 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java
@@ -79,7 +79,7 @@
             }
         }
 
-        GetJobRunJSONWork gjrw = new GetJobRunJSONWork(ccs, jobId);
+        GetJobRunJSONWork gjrw = new GetJobRunJSONWork(ccs.getJobManager(), jobId);
         ccs.getWorkQueue().scheduleAndSync(gjrw);
         Label jobrun = new Label("job-run", gjrw.getJSON().toString());
         jobrun.setEscapeModelStrings(false);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCApplicationContext.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCApplicationContext.java
index dd6f83b..e43a59d 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCApplicationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/application/CCApplicationContext.java
@@ -21,6 +21,7 @@
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -110,7 +111,7 @@
         }
     }
 
-    public void notifyNodeFailure(Set<String> deadNodeIds) throws HyracksException {
+    public void notifyNodeFailure(Collection<String> deadNodeIds) throws HyracksException {
         for (IClusterLifecycleListener l : clusterLifecycleListeners) {
             l.notifyNodeFailure(deadNodeIds);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/INodeManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/INodeManager.java
new file mode 100644
index 0000000..40d81f8
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/INodeManager.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.control.cc.cluster;
+
+import java.net.InetAddress;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hyracks.api.client.NodeControllerInfo;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.control.cc.NodeControllerState;
+
+/**
+ * This interface provides abstractions for a node manager, which manages the node membership in a cluster.
+ */
+public interface INodeManager {
+
+    /**
+     * A functional interface for applying a function for each node.
+     */
+    @FunctionalInterface
+    interface NodeFunction {
+        void apply(String nodeId, NodeControllerState ncState);
+    }
+
+    /**
+     * Applies a function for each node in the cluster.
+     *
+     * @param nodeFunction,
+     *            a function implementation that follows the <code>NodeFunction</code> interface.
+     */
+    void apply(NodeFunction nodeFunction);
+
+    /**
+     * @return all node ids.
+     */
+    Collection<String> getAllNodeIds();
+
+    /**
+     * @return all node controller states.
+     */
+    Collection<NodeControllerState> getAllNodeControllerStates();
+
+    /**
+     * @return the map that maps a IP addresses to a set of node names.
+     */
+    Map<InetAddress, Set<String>> getIpAddressNodeNameMap();
+
+    /**
+     * @return the map that maps a node id to its corresponding node controller info.
+     */
+    Map<String, NodeControllerInfo> getNodeControllerInfoMap();
+
+    /**
+     * Removes all nodes that are considered "dead", i.e., which run out of heartbeats.
+     *
+     * @return all dead nodes and their impacted jobs.
+     * @throws HyracksException
+     *             when any IP address given in the dead nodes is not valid
+     */
+    Pair<Collection<String>, Collection<JobId>> removeDeadNodes() throws HyracksException;
+
+    /**
+     * Retrieves the node controller state from a given node id.
+     *
+     * @param nodeId,
+     *            a given node id.
+     * @return the corresponding node controller state.
+     */
+    NodeControllerState getNodeControllerState(String nodeId);
+
+    /**
+     * Adds one node into the cluster.
+     *
+     * @param nodeId,
+     *            the node id.
+     * @param ncState,
+     *            the node controller state.
+     * @throws HyracksException
+     *             when the node has already been added or the IP address given in the node state is not valid.
+     */
+    void addNode(String nodeId, NodeControllerState ncState) throws HyracksException;
+
+    /**
+     * Removes one node from the cluster.
+     *
+     * @param nodeId,
+     *            the node id.
+     * @throws HyracksException
+     *             when the IP address given in the node state is not valid
+     */
+    void removeNode(String nodeId) throws HyracksException;
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
new file mode 100644
index 0000000..354019c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.control.cc.cluster;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Logger;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hyracks.api.client.NodeControllerInfo;
+import org.apache.hyracks.api.client.NodeStatus;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.resource.NodeCapacity;
+import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.scheduler.IResourceManager;
+import org.apache.hyracks.control.common.controllers.CCConfig;
+
+public class NodeManager implements INodeManager {
+    private static final Logger LOGGER = Logger.getLogger(NodeManager.class.getName());
+
+    private final CCConfig ccConfig;
+    private final IResourceManager resourceManager;
+    private final Map<String, NodeControllerState> nodeRegistry;
+    private final Map<InetAddress, Set<String>> ipAddressNodeNameMap;
+
+    public NodeManager(CCConfig ccConfig, IResourceManager resourceManager) {
+        this.ccConfig = ccConfig;
+        this.resourceManager = resourceManager;
+        this.nodeRegistry = new LinkedHashMap<>();
+        this.ipAddressNodeNameMap = new HashMap<>();
+    }
+
+    @Override
+    public Map<InetAddress, Set<String>> getIpAddressNodeNameMap() {
+        return Collections.unmodifiableMap(ipAddressNodeNameMap);
+    }
+
+    @Override
+    public Collection<String> getAllNodeIds() {
+        return Collections.unmodifiableSet(nodeRegistry.keySet());
+    }
+
+    @Override
+    public Collection<NodeControllerState> getAllNodeControllerStates() {
+        return Collections.unmodifiableCollection(nodeRegistry.values());
+    }
+
+    @Override
+    public NodeControllerState getNodeControllerState(String nodeId) {
+        return nodeRegistry.get(nodeId);
+    }
+
+    @Override
+    public void addNode(String nodeId, NodeControllerState ncState) throws HyracksException {
+        if (nodeId == null || ncState == null) {
+            throw HyracksException.create(ErrorCode.INVALID_INPUT_PARAMETER);
+        }
+        // Updates the node registry.
+        if (nodeRegistry.containsKey(nodeId)) {
+            LOGGER.warning("Node with name " + nodeId + " has already registered.");
+            return;
+        }
+        nodeRegistry.put(nodeId, ncState);
+
+        // Updates the IP address to node names map.
+        try {
+            InetAddress ipAddress = getIpAddress(ncState);
+            Set<String> nodes = ipAddressNodeNameMap.get(ipAddress);
+            if (nodes == null) {
+                nodes = new HashSet<>();
+                ipAddressNodeNameMap.put(ipAddress, nodes);
+            }
+            nodes.add(nodeId);
+        } catch (HyracksException e) {
+            // If anything fails, we ignore the node.
+            nodeRegistry.remove(nodeId);
+            throw e;
+        }
+
+        // Updates the cluster capacity.
+        resourceManager.update(nodeId, ncState.getCapacity());
+    }
+
+    @Override
+    public void removeNode(String nodeId) throws HyracksException {
+        NodeControllerState ncState = nodeRegistry.remove(nodeId);
+        removeNodeFromIpAddressMap(nodeId, ncState);
+
+        // Updates the cluster capacity.
+        resourceManager.update(nodeId, new NodeCapacity(0L, 0));
+    }
+
+    @Override
+    public Map<String, NodeControllerInfo> getNodeControllerInfoMap() {
+        Map<String, NodeControllerInfo> result = new LinkedHashMap<>();
+        for (Map.Entry<String, NodeControllerState> e : nodeRegistry.entrySet()) {
+            NodeControllerState ncState = e.getValue();
+            result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE, ncState.getDataPort(),
+                    ncState.getDatasetPort(), ncState.getMessagingPort(), ncState.getCapacity().getCores()));
+        }
+        return result;
+    }
+
+    @Override
+    public Pair<Collection<String>, Collection<JobId>> removeDeadNodes() throws HyracksException {
+        Set<String> deadNodes = new HashSet<>();
+        Set<JobId> affectedJobIds = new HashSet<>();
+        Iterator<Map.Entry<String, NodeControllerState>> nodeIterator = nodeRegistry.entrySet().iterator();
+        while (nodeIterator.hasNext()) {
+            Map.Entry<String, NodeControllerState> entry = nodeIterator.next();
+            String nodeId = entry.getKey();
+            NodeControllerState state = entry.getValue();
+            if (state.incrementLastHeartbeatDuration() >= ccConfig.maxHeartbeatLapsePeriods) {
+                deadNodes.add(nodeId);
+                affectedJobIds.addAll(state.getActiveJobIds());
+                // Removes the node from node map.
+                nodeIterator.remove();
+                // Removes the node from IP map.
+                removeNodeFromIpAddressMap(nodeId, state);
+                // Updates the cluster capacity.
+                resourceManager.update(nodeId, new NodeCapacity(0L, 0));
+                LOGGER.info(entry.getKey() + " considered dead");
+            }
+        }
+        return Pair.of(deadNodes, affectedJobIds);
+    }
+
+    @Override
+    public void apply(NodeFunction nodeFunction) {
+        nodeRegistry.forEach(nodeFunction::apply);
+    }
+
+    // Removes the entry of the node in <code>ipAddressNodeNameMap</code>.
+    private void removeNodeFromIpAddressMap(String nodeId, NodeControllerState ncState) throws HyracksException {
+        InetAddress ipAddress = getIpAddress(ncState);
+        Set<String> nodes = ipAddressNodeNameMap.get(ipAddress);
+        if (nodes != null) {
+            nodes.remove(nodeId);
+            if (nodes.isEmpty()) {
+                // Removes the ip if no corresponding node exists.
+                ipAddressNodeNameMap.remove(ipAddress);
+            }
+        }
+    }
+
+    // Retrieves the IP address for a given node.
+    private InetAddress getIpAddress(NodeControllerState ncState) throws HyracksException {
+        String ipAddress = ncState.getNCConfig().dataIPAddress;
+        if (ncState.getNCConfig().dataPublicIPAddress != null) {
+            ipAddress = ncState.getNCConfig().dataPublicIPAddress;
+        }
+        try {
+            return InetAddress.getByName(ipAddress);
+        } catch (UnknownHostException e) {
+            throw HyracksException.create(ErrorCode.INVALID_NETWORK_ADDRESS, e, e.getMessage());
+        }
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/ActivityClusterPlanner.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/ActivityClusterPlanner.java
similarity index 74%
rename from hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
rename to hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/ActivityClusterPlanner.java
index c13a458..57b8c50 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/ActivityClusterPlanner.java
@@ -16,7 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.control.cc.scheduler;
+
+package org.apache.hyracks.control.cc.executor;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -30,7 +31,6 @@
 import java.util.logging.Logger;
 
 import org.apache.commons.lang3.tuple.Pair;
-
 import org.apache.hyracks.api.constraints.expressions.LValueConstraintExpression;
 import org.apache.hyracks.api.constraints.expressions.PartitionCountExpression;
 import org.apache.hyracks.api.dataflow.ActivityId;
@@ -52,20 +52,20 @@
 import org.apache.hyracks.control.cc.job.TaskCluster;
 import org.apache.hyracks.control.cc.job.TaskClusterId;
 
-public class ActivityClusterPlanner {
+class ActivityClusterPlanner {
     private static final Logger LOGGER = Logger.getLogger(ActivityClusterPlanner.class.getName());
 
-    private final JobScheduler scheduler;
+    private final JobExecutor executor;
 
     private final Map<PartitionId, TaskCluster> partitionProducingTaskClusterMap;
 
-    public ActivityClusterPlanner(JobScheduler newJobScheduler) {
-        this.scheduler = newJobScheduler;
-        partitionProducingTaskClusterMap = new HashMap<PartitionId, TaskCluster>();
+    ActivityClusterPlanner(JobExecutor newJobExecutor) {
+        this.executor = newJobExecutor;
+        partitionProducingTaskClusterMap = new HashMap<>();
     }
 
-    public ActivityClusterPlan planActivityCluster(ActivityCluster ac) throws HyracksException {
-        JobRun jobRun = scheduler.getJobRun();
+    ActivityClusterPlan planActivityCluster(ActivityCluster ac) throws HyracksException {
+        JobRun jobRun = executor.getJobRun();
         Map<ActivityId, ActivityPartitionDetails> pcMap = computePartitionCounts(ac);
 
         Map<ActivityId, ActivityPlan> activityPlanMap = buildActivityPlanMap(ac, jobRun, pcMap);
@@ -87,8 +87,8 @@
 
     private Map<ActivityId, ActivityPlan> buildActivityPlanMap(ActivityCluster ac, JobRun jobRun,
             Map<ActivityId, ActivityPartitionDetails> pcMap) {
-        Map<ActivityId, ActivityPlan> activityPlanMap = new HashMap<ActivityId, ActivityPlan>();
-        Set<ActivityId> depAnIds = new HashSet<ActivityId>();
+        Map<ActivityId, ActivityPlan> activityPlanMap = new HashMap<>();
+        Set<ActivityId> depAnIds = new HashSet<>();
         for (ActivityId anId : ac.getActivityMap().keySet()) {
             depAnIds.clear();
             getDependencyActivityIds(depAnIds, anId, ac);
@@ -101,13 +101,15 @@
                 for (ActivityId danId : depAnIds) {
                     ActivityCluster dAC = ac.getActivityClusterGraph().getActivityMap().get(danId);
                     ActivityClusterPlan dACP = jobRun.getActivityClusterPlanMap().get(dAC.getId());
-                    assert dACP != null : "IllegalStateEncountered: Dependent AC is being planned without a plan for dependency AC: Encountered no plan for ActivityID "
+                    assert dACP != null : "IllegalStateEncountered: Dependent AC is being planned without a plan for "
+                            + "dependency AC: Encountered no plan for ActivityID "
                             + danId;
                     Task[] dATasks = dACP.getActivityPlanMap().get(danId).getTasks();
-                    assert dATasks != null : "IllegalStateEncountered: Dependent AC is being planned without a plan for dependency AC: Encountered no plan for ActivityID "
+                    assert dATasks != null : "IllegalStateEncountered: Dependent AC is being planned without a plan for"
+                            + " dependency AC: Encountered no plan for ActivityID "
                             + danId;
-                    assert dATasks.length == tasks.length : "Dependency activity partitioned differently from dependent: "
-                            + dATasks.length + " != " + tasks.length;
+                    assert dATasks.length == tasks.length : "Dependency activity partitioned differently from "
+                            + "dependent: " + dATasks.length + " != " + tasks.length;
                     Task dTask = dATasks[i];
                     TaskId dTaskId = dTask.getTaskId();
                     tasks[i].getDependencies().add(dTaskId);
@@ -126,8 +128,9 @@
         Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> taskConnectivity = computeTaskConnectivity(jobRun,
                 activityPlanMap, activities);
 
-        TaskCluster[] taskClusters = ac.getActivityClusterGraph().isUseConnectorPolicyForScheduling() ? buildConnectorPolicyAwareTaskClusters(
-                ac, activityPlanMap, taskConnectivity) : buildConnectorPolicyUnawareTaskClusters(ac, activityPlanMap);
+        TaskCluster[] taskClusters = ac.getActivityClusterGraph().isUseConnectorPolicyForScheduling()
+                ? buildConnectorPolicyAwareTaskClusters(ac, activityPlanMap, taskConnectivity)
+                : buildConnectorPolicyUnawareTaskClusters(ac, activityPlanMap);
 
         for (TaskCluster tc : taskClusters) {
             Set<TaskCluster> tcDependencyTaskClusters = tc.getDependencyTaskClusters();
@@ -149,7 +152,6 @@
                         }
                     }
                 }
-
                 for (TaskId dTid : ts.getDependencies()) {
                     TaskCluster dTC = getTaskCluster(dTid);
                     dTC.getDependentTaskClusters().add(tc);
@@ -162,13 +164,11 @@
 
     private TaskCluster[] buildConnectorPolicyUnawareTaskClusters(ActivityCluster ac,
             Map<ActivityId, ActivityPlan> activityPlanMap) {
-        List<Task> taskStates = new ArrayList<Task>();
+        List<Task> taskStates = new ArrayList<>();
         for (ActivityId anId : ac.getActivityMap().keySet()) {
             ActivityPlan ap = activityPlanMap.get(anId);
             Task[] tasks = ap.getTasks();
-            for (Task t : tasks) {
-                taskStates.add(t);
-            }
+            taskStates.addAll(Arrays.asList(tasks));
         }
         TaskCluster tc = new TaskCluster(new TaskClusterId(ac.getId(), 0), ac, taskStates.toArray(new Task[taskStates
                 .size()]));
@@ -180,7 +180,7 @@
 
     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>>>();
+        Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> taskConnectivity = new HashMap<>();
         ActivityClusterGraph acg = jobRun.getActivityClusterGraph();
         BitSet targetBitmap = new BitSet();
         for (ActivityId ac1 : activities) {
@@ -188,35 +188,36 @@
             Task[] ac1TaskStates = activityPlanMap.get(ac1).getTasks();
             int nProducers = ac1TaskStates.length;
             List<IConnectorDescriptor> outputConns = ac.getActivityOutputMap().get(ac1);
-            if (outputConns != null) {
-                for (IConnectorDescriptor c : outputConns) {
-                    ConnectorDescriptorId cdId = c.getConnectorId();
-                    ActivityId ac2 = ac.getConsumerActivity(cdId);
-                    Task[] ac2TaskStates = activityPlanMap.get(ac2).getTasks();
-                    int nConsumers = ac2TaskStates.length;
-                    if (c.allProducersToAllConsumers()) {
-                        List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = new ArrayList<Pair<TaskId, ConnectorDescriptorId>>();
-                        for (int j = 0; j < nConsumers; j++) {
-                            TaskId targetTID = ac2TaskStates[j].getTaskId();
-                            cInfoList.add(Pair.<TaskId, ConnectorDescriptorId> of(targetTID, cdId));
-                        }
-                        for (int i = 0; i < nProducers; ++i) {
-                            taskConnectivity.put(ac1TaskStates[i].getTaskId(), cInfoList);
-                        }
-                    } else {
-                        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));
-                            }
-                        }
+            if (outputConns == null) {
+                continue;
+            }
+            for (IConnectorDescriptor c : outputConns) {
+                ConnectorDescriptorId cdId = c.getConnectorId();
+                ActivityId ac2 = ac.getConsumerActivity(cdId);
+                Task[] ac2TaskStates = activityPlanMap.get(ac2).getTasks();
+                int nConsumers = ac2TaskStates.length;
+                if (c.allProducersToAllConsumers()) {
+                    List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = new ArrayList<>();
+                    for (int j = 0; j < nConsumers; j++) {
+                        TaskId targetTID = ac2TaskStates[j].getTaskId();
+                        cInfoList.add(Pair.of(targetTID, cdId));
+                    }
+                    for (int i = 0; i < nProducers; ++i) {
+                        taskConnectivity.put(ac1TaskStates[i].getTaskId(), cInfoList);
+                    }
+                    continue;
+                }
+                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<>();
+                        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.of(targetTID, cdId));
                     }
                 }
             }
@@ -227,19 +228,19 @@
     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>>();
+        Map<TaskId, Set<TaskId>> taskClusterMap = new HashMap<>();
         for (ActivityId anId : ac.getActivityMap().keySet()) {
             ActivityPlan ap = activityPlanMap.get(anId);
             Task[] tasks = ap.getTasks();
             for (Task t : tasks) {
-                Set<TaskId> cluster = new HashSet<TaskId>();
+                Set<TaskId> cluster = new HashSet<>();
                 TaskId tid = t.getTaskId();
                 cluster.add(tid);
                 taskClusterMap.put(tid, cluster);
             }
         }
 
-        JobRun jobRun = scheduler.getJobRun();
+        JobRun jobRun = executor.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());
@@ -252,13 +253,12 @@
         }
 
         /*
-         * 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
-         * tasks that need to be co-scheduled
+         * We compute the transitive closure of this (producer-consumer) relation to find the largest set of
+         * tasks that need to be co-scheduled.
          */
         int counter = 0;
         TaskId[] ordinalList = new TaskId[taskClusterMap.size()];
-        Map<TaskId, Integer> ordinalMap = new HashMap<TaskId, Integer>();
+        Map<TaskId, Integer> ordinalMap = new HashMap<>();
         for (TaskId tid : taskClusterMap.keySet()) {
             ordinalList[counter] = tid;
             ordinalMap.put(tid, counter);
@@ -295,9 +295,9 @@
         }
         BitSet pending = new BitSet(n);
         pending.set(0, n);
-        List<List<TaskId>> clusters = new ArrayList<List<TaskId>>();
+        List<List<TaskId>> clusters = new ArrayList<>();
         for (int i = pending.nextSetBit(0); i >= 0; i = pending.nextSetBit(i)) {
-            List<TaskId> cluster = new ArrayList<TaskId>();
+            List<TaskId> cluster = new ArrayList<>();
             for (int j = paths[i].nextSetBit(0); j >= 0; j = paths[i].nextSetBit(j + 1)) {
                 cluster.add(ordinalList[j]);
                 pending.clear(j);
@@ -305,10 +305,10 @@
             clusters.add(cluster);
         }
 
-        List<TaskCluster> tcSet = new ArrayList<TaskCluster>();
+        List<TaskCluster> tcSet = new ArrayList<>();
         counter = 0;
         for (List<TaskId> cluster : clusters) {
-            List<Task> taskStates = new ArrayList<Task>();
+            List<Task> taskStates = new ArrayList<>();
             for (TaskId tid : cluster) {
                 taskStates.add(activityPlanMap.get(tid.getActivityId()).getTasks()[tid.getPartition()]);
             }
@@ -319,12 +319,11 @@
                 activityPlanMap.get(tid.getActivityId()).getTasks()[tid.getPartition()].setTaskCluster(tc);
             }
         }
-        TaskCluster[] taskClusters = tcSet.toArray(new TaskCluster[tcSet.size()]);
-        return taskClusters;
+        return tcSet.toArray(new TaskCluster[tcSet.size()]);
     }
 
     private TaskCluster getTaskCluster(TaskId tid) {
-        JobRun run = scheduler.getJobRun();
+        JobRun run = executor.getJobRun();
         ActivityCluster ac = run.getActivityClusterGraph().getActivityMap().get(tid.getActivityId());
         ActivityClusterPlan acp = run.getActivityClusterPlanMap().get(ac.getId());
         Task[] tasks = acp.getActivityPlanMap().get(tid.getActivityId()).getTasks();
@@ -341,37 +340,38 @@
     }
 
     private void assignConnectorPolicy(ActivityCluster ac, Map<ActivityId, ActivityPlan> taskMap) {
-        Map<ConnectorDescriptorId, IConnectorPolicy> cPolicyMap = new HashMap<ConnectorDescriptorId, IConnectorPolicy>();
+        Map<ConnectorDescriptorId, IConnectorPolicy> cPolicyMap = new HashMap<>();
         Set<ActivityId> activities = ac.getActivityMap().keySet();
         BitSet targetBitmap = new BitSet();
         for (ActivityId a1 : activities) {
             Task[] ac1TaskStates = taskMap.get(a1).getTasks();
             int nProducers = ac1TaskStates.length;
             List<IConnectorDescriptor> outputConns = ac.getActivityOutputMap().get(a1);
-            if (outputConns != null) {
-                for (IConnectorDescriptor c : outputConns) {
-                    ConnectorDescriptorId cdId = c.getConnectorId();
-                    ActivityId a2 = ac.getConsumerActivity(cdId);
-                    Task[] ac2TaskStates = taskMap.get(a2).getTasks();
-                    int nConsumers = ac2TaskStates.length;
+            if (outputConns == null) {
+                continue;
+            }
+            for (IConnectorDescriptor c : outputConns) {
+                ConnectorDescriptorId cdId = c.getConnectorId();
+                ActivityId a2 = ac.getConsumerActivity(cdId);
+                Task[] ac2TaskStates = taskMap.get(a2).getTasks();
+                int nConsumers = ac2TaskStates.length;
 
-                    int[] fanouts = new int[nProducers];
-                    if (c.allProducersToAllConsumers()) {
+                int[] fanouts = new int[nProducers];
+                if (c.allProducersToAllConsumers()) {
                         for (int i = 0; i < nProducers; ++i) {
                             fanouts[i] = nConsumers;
                         }
-                    } else {
-                        for (int i = 0; i < nProducers; ++i) {
-                            c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
-                            fanouts[i] = targetBitmap.cardinality();
-                        }
+                } else {
+                    for (int i = 0; i < nProducers; ++i) {
+                        c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
+                        fanouts[i] = targetBitmap.cardinality();
                     }
-                    IConnectorPolicy cp = assignConnectorPolicy(ac, c, nProducers, nConsumers, fanouts);
-                    cPolicyMap.put(cdId, cp);
                 }
+                IConnectorPolicy cp = assignConnectorPolicy(ac, c, nProducers, nConsumers, fanouts);
+                cPolicyMap.put(cdId, cp);
             }
         }
-        scheduler.getJobRun().getConnectorPolicyMap().putAll(cPolicyMap);
+        executor.getJobRun().getConnectorPolicyMap().putAll(cPolicyMap);
     }
 
     private IConnectorPolicy assignConnectorPolicy(ActivityCluster ac, IConnectorDescriptor c, int nProducers,
@@ -389,13 +389,13 @@
 
     private Map<ActivityId, ActivityPartitionDetails> computePartitionCounts(ActivityCluster ac)
             throws HyracksException {
-        PartitionConstraintSolver solver = scheduler.getSolver();
-        Set<LValueConstraintExpression> lValues = new HashSet<LValueConstraintExpression>();
+        PartitionConstraintSolver solver = executor.getSolver();
+        Set<LValueConstraintExpression> lValues = new HashSet<>();
         for (ActivityId anId : ac.getActivityMap().keySet()) {
             lValues.add(new PartitionCountExpression(anId.getOperatorDescriptorId()));
         }
         solver.solve(lValues);
-        Map<OperatorDescriptorId, Integer> nPartMap = new HashMap<OperatorDescriptorId, Integer>();
+        Map<OperatorDescriptorId, Integer> nPartMap = new HashMap<>();
         for (LValueConstraintExpression lv : lValues) {
             Object value = solver.getValue(lv);
             if (value == null) {
@@ -409,9 +409,9 @@
             if (nParts <= 0) {
                 throw new HyracksException("Unsatisfiable number of partitions for " + lv + ": " + nParts);
             }
-            nPartMap.put(((PartitionCountExpression) lv).getOperatorDescriptorId(), Integer.valueOf(nParts));
+            nPartMap.put(((PartitionCountExpression) lv).getOperatorDescriptorId(), nParts);
         }
-        Map<ActivityId, ActivityPartitionDetails> activityPartsMap = new HashMap<ActivityId, ActivityPartitionDetails>();
+        Map<ActivityId, ActivityPartitionDetails> activityPartsMap = new HashMap<>();
         for (ActivityId anId : ac.getActivityMap().keySet()) {
             int nParts = nPartMap.get(anId.getOperatorDescriptorId());
             int[] nInputPartitions = null;
@@ -442,7 +442,7 @@
         return activityPartsMap;
     }
 
-    public Map<? extends PartitionId, ? extends TaskCluster> getPartitionProducingTaskClusterMap() {
+    Map<PartitionId, TaskCluster> getPartitionProducingTaskClusterMap() {
         return partitionProducingTaskClusterMap;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/ActivityPartitionDetails.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/ActivityPartitionDetails.java
similarity index 97%
rename from hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/ActivityPartitionDetails.java
rename to hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/ActivityPartitionDetails.java
index 97b459c..60f6e88 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/ActivityPartitionDetails.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/ActivityPartitionDetails.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.control.cc.scheduler;
+package org.apache.hyracks.control.cc.executor;
 
 import java.util.Arrays;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
similarity index 75%
rename from hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/JobScheduler.java
rename to hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
index b577ff7..3eece52 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/JobScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/JobExecutor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.control.cc.scheduler;
+package org.apache.hyracks.control.cc.executor;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -31,8 +31,8 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.constraints.Constraint;
 import org.apache.hyracks.api.constraints.expressions.LValueConstraintExpression;
@@ -54,7 +54,6 @@
 import org.apache.hyracks.api.util.JavaSerializationUtils;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
-import org.apache.hyracks.control.cc.application.CCApplicationContext;
 import org.apache.hyracks.control.cc.job.ActivityClusterPlan;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.cc.job.Task;
@@ -66,8 +65,8 @@
 import org.apache.hyracks.control.common.job.PartitionState;
 import org.apache.hyracks.control.common.job.TaskAttemptDescriptor;
 
-public class JobScheduler {
-    private static final Logger LOGGER = Logger.getLogger(JobScheduler.class.getName());
+public class JobExecutor {
+    private static final Logger LOGGER = Logger.getLogger(JobExecutor.class.getName());
 
     private final ClusterControllerService ccs;
 
@@ -79,14 +78,16 @@
 
     private final Set<TaskCluster> inProgressTaskClusters;
 
+    private final Random random;
 
-    public JobScheduler(ClusterControllerService ccs, JobRun jobRun, Collection<Constraint> constraints) {
+    public JobExecutor(ClusterControllerService ccs, JobRun jobRun, Collection<Constraint> constraints) {
         this.ccs = ccs;
         this.jobRun = jobRun;
         solver = new PartitionConstraintSolver();
         partitionProducingTaskClusterMap = new HashMap<PartitionId, TaskCluster>();
         inProgressTaskClusters = new HashSet<TaskCluster>();
         solver.addConstraints(constraints);
+        random = new Random();
     }
 
     public JobRun getJobRun() {
@@ -119,12 +120,13 @@
             } else {
                 boolean tcRootsComplete = true;
                 for (TaskCluster tc : getActivityClusterPlan(depAC).getTaskClusters()) {
-                    if (tc.getProducedPartitions().isEmpty()) {
-                        TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
-                        if (tca == null || tca.getStatus() != TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
-                            tcRootsComplete = false;
-                            break;
-                        }
+                    if (!tc.getProducedPartitions().isEmpty()) {
+                        continue;
+                    }
+                    TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
+                    if (tca == null || tca.getStatus() != TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
+                        tcRootsComplete = false;
+                        break;
                     }
                 }
                 if (!tcRootsComplete) {
@@ -133,20 +135,22 @@
                 }
             }
         }
-        if (depsComplete) {
-            if (!isPlanned(candidate)) {
-                ActivityClusterPlanner acp = new ActivityClusterPlanner(this);
-                ActivityClusterPlan acPlan = acp.planActivityCluster(candidate);
-                jobRun.getActivityClusterPlanMap().put(candidate.getId(), acPlan);
-                partitionProducingTaskClusterMap.putAll(acp.getPartitionProducingTaskClusterMap());
+        if (!depsComplete) {
+            return;
+        }
+        if (!isPlanned(candidate)) {
+            ActivityClusterPlanner acp = new ActivityClusterPlanner(this);
+            ActivityClusterPlan acPlan = acp.planActivityCluster(candidate);
+            jobRun.getActivityClusterPlanMap().put(candidate.getId(), acPlan);
+            partitionProducingTaskClusterMap.putAll(acp.getPartitionProducingTaskClusterMap());
+        }
+        for (TaskCluster tc : getActivityClusterPlan(candidate).getTaskClusters()) {
+            if (!tc.getProducedPartitions().isEmpty()) {
+                continue;
             }
-            for (TaskCluster tc : getActivityClusterPlan(candidate).getTaskClusters()) {
-                if (tc.getProducedPartitions().isEmpty()) {
-                    TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
-                    if (tca == null || tca.getStatus() != TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
-                        frontier.add(tc);
-                    }
-                }
+            TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
+            if (tca == null || tca.getStatus() != TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
+                frontier.add(tc);
             }
         }
     }
@@ -160,7 +164,7 @@
     }
 
     private void startRunnableActivityClusters() throws HyracksException {
-        Set<TaskCluster> taskClusterRoots = new HashSet<TaskCluster>();
+        Set<TaskCluster> taskClusterRoots = new HashSet<>();
         findRunnableTaskClusterRoots(taskClusterRoots, jobRun.getActivityClusterGraph().getActivityClusterMap()
                 .values());
         if (LOGGER.isLoggable(Level.FINE)) {
@@ -168,19 +172,20 @@
                     + inProgressTaskClusters);
         }
         if (taskClusterRoots.isEmpty() && inProgressTaskClusters.isEmpty()) {
-            ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, jobRun.getJobId(), JobStatus.TERMINATED, null));
+            ccs.getWorkQueue()
+                    .schedule(new JobCleanupWork(ccs.getJobManager(), jobRun.getJobId(), JobStatus.TERMINATED, null));
             return;
         }
         startRunnableTaskClusters(taskClusterRoots);
     }
 
     private void startRunnableTaskClusters(Set<TaskCluster> tcRoots) throws HyracksException {
-        Map<TaskCluster, Runnability> runnabilityMap = new HashMap<TaskCluster, Runnability>();
+        Map<TaskCluster, Runnability> runnabilityMap = new HashMap<>();
         for (TaskCluster tc : tcRoots) {
             assignRunnabilityRank(tc, runnabilityMap);
         }
 
-        PriorityQueue<RankedRunnableTaskCluster> queue = new PriorityQueue<RankedRunnableTaskCluster>();
+        PriorityQueue<RankedRunnableTaskCluster> queue = new PriorityQueue<>();
         for (Map.Entry<TaskCluster, Runnability> e : runnabilityMap.entrySet()) {
             TaskCluster tc = e.getKey();
             Runnability runnability = e.getValue();
@@ -196,7 +201,7 @@
             LOGGER.fine("Ranked TCs: " + queue);
         }
 
-        Map<String, List<TaskAttemptDescriptor>> taskAttemptMap = new HashMap<String, List<TaskAttemptDescriptor>>();
+        Map<String, List<TaskAttemptDescriptor>> taskAttemptMap = new HashMap<>();
         for (RankedRunnableTaskCluster rrtc : queue) {
             TaskCluster tc = rrtc.getTaskCluster();
             if (LOGGER.isLoggable(Level.FINE)) {
@@ -285,6 +290,8 @@
                             runnability = new Runnability(Runnability.Tag.RUNNABLE, 1);
                         }
                         break;
+                    default:
+                        break;
                 }
             }
             aggregateRunnability = Runnability.getWorstCase(aggregateRunnability, runnability);
@@ -307,8 +314,8 @@
         List<TaskClusterAttempt> tcAttempts = tc.getAttempts();
         int attempts = tcAttempts.size();
         TaskClusterAttempt tcAttempt = new TaskClusterAttempt(tc, attempts);
-        Map<TaskId, TaskAttempt> taskAttempts = new HashMap<TaskId, TaskAttempt>();
-        Map<TaskId, LValueConstraintExpression> locationMap = new HashMap<TaskId, LValueConstraintExpression>();
+        Map<TaskId, TaskAttempt> taskAttempts = new HashMap<>();
+        Map<TaskId, LValueConstraintExpression> locationMap = new HashMap<>();
         for (int i = 0; i < tasks.length; ++i) {
             Task ts = tasks[i];
             TaskId tid = ts.getTaskId();
@@ -331,7 +338,7 @@
             taskAttempt.setStartTime(System.currentTimeMillis());
             List<TaskAttemptDescriptor> tads = taskAttemptMap.get(nodeId);
             if (tads == null) {
-                tads = new ArrayList<TaskAttemptDescriptor>();
+                tads = new ArrayList<>();
                 taskAttemptMap.put(nodeId, tads);
             }
             OperatorDescriptorId opId = tid.getActivityId().getOperatorDescriptorId();
@@ -349,6 +356,7 @@
          * we set the NetworkAddress[][] partitionLocations, in which each row is for an incoming connector descriptor
          * and each column is for an input channel of the connector.
          */
+        INodeManager nodeManager = ccs.getNodeManager();
         for (Map.Entry<String, List<TaskAttemptDescriptor>> e : taskAttemptMap.entrySet()) {
             List<TaskAttemptDescriptor> tads = e.getValue();
             for (TaskAttemptDescriptor tad : tads) {
@@ -358,29 +366,30 @@
                 ActivityId aid = tid.getActivityId();
                 List<IConnectorDescriptor> inConnectors = acg.getActivityInputs(aid);
                 int[] inPartitionCounts = tad.getInputPartitionCounts();
-                if (inPartitionCounts != null) {
-                    NetworkAddress[][] partitionLocations = new NetworkAddress[inPartitionCounts.length][];
-                    for (int i = 0; i < inPartitionCounts.length; ++i) {
-                        ConnectorDescriptorId cdId = inConnectors.get(i).getConnectorId();
-                        IConnectorPolicy policy = jobRun.getConnectorPolicyMap().get(cdId);
-                        /**
-                         * carry sender location information into a task
-                         * when it is not the case that it is an re-attempt and the send-side
-                         * is materialized blocking.
-                         */
-                        if (!(attempt > 0 && policy.materializeOnSendSide() && policy
-                                .consumerWaitsForProducerToFinish())) {
-                            ActivityId producerAid = acg.getProducerActivity(cdId);
-                            partitionLocations[i] = new NetworkAddress[inPartitionCounts[i]];
-                            for (int j = 0; j < inPartitionCounts[i]; ++j) {
-                                TaskId producerTaskId = new TaskId(producerAid, j);
-                                String nodeId = findTaskLocation(producerTaskId);
-                                partitionLocations[i][j] = ccs.getNodeMap().get(nodeId).getDataPort();
-                            }
-                        }
-                    }
-                    tad.setInputPartitionLocations(partitionLocations);
+                if (inPartitionCounts == null) {
+                    continue;
                 }
+                NetworkAddress[][] partitionLocations = new NetworkAddress[inPartitionCounts.length][];
+                for (int i = 0; i < inPartitionCounts.length; ++i) {
+                    ConnectorDescriptorId cdId = inConnectors.get(i).getConnectorId();
+                    IConnectorPolicy policy = jobRun.getConnectorPolicyMap().get(cdId);
+                    /**
+                     * carry sender location information into a task
+                     * when it is not the case that it is an re-attempt and the send-side
+                     * is materialized blocking.
+                     */
+                    if (attempt > 0 && policy.materializeOnSendSide() && policy.consumerWaitsForProducerToFinish()) {
+                        continue;
+                    }
+                    ActivityId producerAid = acg.getProducerActivity(cdId);
+                    partitionLocations[i] = new NetworkAddress[inPartitionCounts[i]];
+                    for (int j = 0; j < inPartitionCounts[i]; ++j) {
+                        TaskId producerTaskId = new TaskId(producerAid, j);
+                        String nodeId = findTaskLocation(producerTaskId);
+                        partitionLocations[i][j] = nodeManager.getNodeControllerState(nodeId).getDataPort();
+                    }
+                }
+                tad.setInputPartitionLocations(partitionLocations);
             }
         }
 
@@ -403,14 +412,14 @@
                 }
             }
         }
-        Set<String> liveNodes = ccs.getNodeMap().keySet();
+        INodeManager nodeManager = ccs.getNodeManager();
+        Collection<String> liveNodes = nodeManager.getAllNodeIds();
         if (nodeId == null) {
             LValueConstraintExpression pLocationExpr = locationMap.get(tid);
             Object location = solver.getValue(pLocationExpr);
             if (location == null) {
                 // pick any
-                nodeId = liveNodes.toArray(new String[liveNodes.size()])[Math.abs(new Random().nextInt())
-                        % liveNodes.size()];
+                nodeId = liveNodes.toArray(new String[liveNodes.size()])[random.nextInt(1) % liveNodes.size()];
             } else if (location instanceof String) {
                 nodeId = (String) location;
             } else if (location instanceof String[]) {
@@ -462,14 +471,15 @@
         final DeploymentId deploymentId = jobRun.getDeploymentId();
         final JobId jobId = jobRun.getJobId();
         final ActivityClusterGraph acg = jobRun.getActivityClusterGraph();
-        final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = new HashMap<ConnectorDescriptorId, IConnectorPolicy>(
+        final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = new HashMap<>(
                 jobRun.getConnectorPolicyMap());
+        INodeManager nodeManager = ccs.getNodeManager();
         try {
             byte[] acgBytes = JavaSerializationUtils.serialize(acg);
             for (Map.Entry<String, List<TaskAttemptDescriptor>> entry : taskAttemptMap.entrySet()) {
                 String nodeId = entry.getKey();
                 final List<TaskAttemptDescriptor> taskDescriptors = entry.getValue();
-                final NodeControllerState node = ccs.getNodeMap().get(nodeId);
+                final NodeControllerState node = nodeManager.getNodeControllerState(nodeId);
                 if (node != null) {
                     node.getActiveJobIds().add(jobRun.getJobId());
                     boolean changed = jobRun.getParticipatingNodeIds().add(nodeId);
@@ -487,19 +497,20 @@
     }
 
     private void abortJob(List<Exception> exceptions) {
-        Set<TaskCluster> inProgressTaskClustersCopy = new HashSet<TaskCluster>(inProgressTaskClusters);
+        Set<TaskCluster> inProgressTaskClustersCopy = new HashSet<>(inProgressTaskClusters);
         for (TaskCluster tc : inProgressTaskClustersCopy) {
             abortTaskCluster(findLastTaskClusterAttempt(tc), TaskClusterAttempt.TaskClusterStatus.ABORTED);
         }
         assert inProgressTaskClusters.isEmpty();
-        ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, jobRun.getJobId(), JobStatus.FAILURE, exceptions));
+        ccs.getWorkQueue()
+                .schedule(new JobCleanupWork(ccs.getJobManager(), jobRun.getJobId(), JobStatus.FAILURE, exceptions));
     }
 
     private void abortTaskCluster(TaskClusterAttempt tcAttempt,
             TaskClusterAttempt.TaskClusterStatus failedOrAbortedStatus) {
         LOGGER.fine("Aborting task cluster: " + tcAttempt.getAttempt());
-        Set<TaskAttemptId> abortTaskIds = new HashSet<TaskAttemptId>();
-        Map<String, List<TaskAttemptId>> abortTaskAttemptMap = new HashMap<String, List<TaskAttemptId>>();
+        Set<TaskAttemptId> abortTaskIds = new HashSet<>();
+        Map<String, List<TaskAttemptId>> abortTaskAttemptMap = new HashMap<>();
         for (TaskAttempt ta : tcAttempt.getTaskAttempts().values()) {
             TaskAttemptId taId = ta.getTaskAttemptId();
             TaskAttempt.TaskStatus status = ta.getStatus();
@@ -510,7 +521,7 @@
                 ta.setEndTime(System.currentTimeMillis());
                 List<TaskAttemptId> abortTaskAttempts = abortTaskAttemptMap.get(ta.getNodeId());
                 if (status == TaskAttempt.TaskStatus.RUNNING && abortTaskAttempts == null) {
-                    abortTaskAttempts = new ArrayList<TaskAttemptId>();
+                    abortTaskAttempts = new ArrayList<>();
                     abortTaskAttemptMap.put(ta.getNodeId(), abortTaskAttempts);
                 }
                 if (status == TaskAttempt.TaskStatus.RUNNING) {
@@ -520,8 +531,9 @@
         }
         final JobId jobId = jobRun.getJobId();
         LOGGER.fine("Abort map for job: " + jobId + ": " + abortTaskAttemptMap);
+        INodeManager nodeManager = ccs.getNodeManager();
         for (Map.Entry<String, List<TaskAttemptId>> entry : abortTaskAttemptMap.entrySet()) {
-            final NodeControllerState node = ccs.getNodeMap().get(entry.getKey());
+            final NodeControllerState node = nodeManager.getNodeControllerState(entry.getKey());
             final List<TaskAttemptId> abortTaskAttempts = entry.getValue();
             if (node != null) {
                 if (LOGGER.isLoggable(Level.FINE)) {
@@ -530,7 +542,7 @@
                 try {
                     node.getNodeController().abortTasks(jobId, abortTaskAttempts);
                 } catch (Exception e) {
-                    e.printStackTrace();
+                    LOGGER.log(Level.SEVERE, e.getMessage(), e);
                 }
             }
         }
@@ -545,7 +557,7 @@
     }
 
     private void abortDoomedTaskClusters() throws HyracksException {
-        Set<TaskCluster> doomedTaskClusters = new HashSet<TaskCluster>();
+        Set<TaskCluster> doomedTaskClusters = new HashSet<>();
         for (TaskCluster tc : inProgressTaskClusters) {
             // Start search at TCs that produce no outputs (sinks)
             if (tc.getProducedPartitions().isEmpty()) {
@@ -570,10 +582,10 @@
             switch (lastAttempt.getStatus()) {
                 case ABORTED:
                 case FAILED:
-                    return true;
-
                 case COMPLETED:
                     return false;
+                default:
+                    break;
             }
         }
         Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicyMap = jobRun.getConnectorPolicyMap();
@@ -588,11 +600,10 @@
             ConnectorDescriptorId cdId = pid.getConnectorDescriptorId();
             IConnectorPolicy cPolicy = connectorPolicyMap.get(cdId);
             PartitionState maxState = pmm.getMaximumAvailableState(pid);
-            if (maxState == null
-                    || (cPolicy.consumerWaitsForProducerToFinish() && maxState != PartitionState.COMMITTED)) {
-                if (findDoomedTaskClusters(partitionProducingTaskClusterMap.get(pid), doomedTaskClusters)) {
+            if ((maxState == null
+                    || (cPolicy.consumerWaitsForProducerToFinish() && maxState != PartitionState.COMMITTED))
+                    && findDoomedTaskClusters(partitionProducingTaskClusterMap.get(pid), doomedTaskClusters)) {
                     doomed = true;
-                }
             }
         }
         if (doomed) {
@@ -605,22 +616,23 @@
         TaskAttemptId taId = ta.getTaskAttemptId();
         TaskCluster tc = ta.getTask().getTaskCluster();
         TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
-        if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
-            TaskAttempt.TaskStatus taStatus = ta.getStatus();
-            if (taStatus == TaskAttempt.TaskStatus.RUNNING) {
-                ta.setStatus(TaskAttempt.TaskStatus.COMPLETED, null);
-                ta.setEndTime(System.currentTimeMillis());
-                if (lastAttempt.decrementPendingTasksCounter() == 0) {
-                    lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.COMPLETED);
-                    lastAttempt.setEndTime(System.currentTimeMillis());
-                    inProgressTaskClusters.remove(tc);
-                    startRunnableActivityClusters();
-                }
-            } else {
-                LOGGER.warning("Spurious task complete notification: " + taId + " Current state = " + taStatus);
-            }
-        } else {
-            LOGGER.warning("Ignoring task complete notification: " + taId + " -- Current last attempt = " + lastAttempt);
+        if (lastAttempt == null || taId.getAttempt() != lastAttempt.getAttempt()) {
+            LOGGER.warning(
+                    "Ignoring task complete notification: " + taId + " -- Current last attempt = " + lastAttempt);
+            return;
+        }
+        TaskAttempt.TaskStatus taStatus = ta.getStatus();
+        if (taStatus != TaskAttempt.TaskStatus.RUNNING) {
+            LOGGER.warning("Spurious task complete notification: " + taId + " Current state = " + taStatus);
+            return;
+        }
+        ta.setStatus(TaskAttempt.TaskStatus.COMPLETED, null);
+        ta.setEndTime(System.currentTimeMillis());
+        if (lastAttempt.decrementPendingTasksCounter() == 0) {
+            lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.COMPLETED);
+            lastAttempt.setEndTime(System.currentTimeMillis());
+            inProgressTaskClusters.remove(tc);
+            startRunnableActivityClusters();
         }
     }
 
@@ -660,45 +672,47 @@
      * @param deadNodes
      *            - Set of failed nodes
      */
-    public void notifyNodeFailures(Set<String> deadNodes) {
+    public void notifyNodeFailures(Collection<String> deadNodes) {
         try {
             jobRun.getPartitionMatchMaker().notifyNodeFailures(deadNodes);
             jobRun.getParticipatingNodeIds().removeAll(deadNodes);
             jobRun.getCleanupPendingNodeIds().removeAll(deadNodes);
             if (jobRun.getPendingStatus() != null && jobRun.getCleanupPendingNodeIds().isEmpty()) {
-                finishJob(jobRun);
+                IJobManager jobManager = ccs.getJobManager();
+                jobManager.finalComplete(jobRun);
                 return;
             }
             for (ActivityCluster ac : jobRun.getActivityClusterGraph().getActivityClusterMap().values()) {
-                if (isPlanned(ac)) {
-                    TaskCluster[] taskClusters = getActivityClusterPlan(ac).getTaskClusters();
-                    if (taskClusters != null) {
-                        for (TaskCluster tc : taskClusters) {
-                            TaskClusterAttempt lastTaskClusterAttempt = findLastTaskClusterAttempt(tc);
-                            if (lastTaskClusterAttempt != null
-                                    && (lastTaskClusterAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.COMPLETED || lastTaskClusterAttempt
-                                            .getStatus() == TaskClusterAttempt.TaskClusterStatus.RUNNING)) {
-                                boolean abort = false;
-                                for (TaskAttempt ta : lastTaskClusterAttempt.getTaskAttempts().values()) {
-                                    assert (ta.getStatus() == TaskAttempt.TaskStatus.COMPLETED || ta.getStatus() == TaskAttempt.TaskStatus.RUNNING);
-                                    if (deadNodes.contains(ta.getNodeId())) {
-                                        ta.setStatus(
-                                                TaskAttempt.TaskStatus.FAILED,
-                                                Collections.singletonList(new Exception("Node " + ta.getNodeId()
-                                                        + " failed")));
-                                        ta.setEndTime(System.currentTimeMillis());
-                                        abort = true;
-                                    }
-                                }
-                                if (abort) {
-                                    abortTaskCluster(lastTaskClusterAttempt,
-                                            TaskClusterAttempt.TaskClusterStatus.ABORTED);
-                                }
-                            }
+                if (!isPlanned(ac)) {
+                    continue;
+                }
+                TaskCluster[] taskClusters = getActivityClusterPlan(ac).getTaskClusters();
+                if (taskClusters == null) {
+                    continue;
+                }
+                for (TaskCluster tc : taskClusters) {
+                    TaskClusterAttempt lastTaskClusterAttempt = findLastTaskClusterAttempt(tc);
+                    if (lastTaskClusterAttempt == null || !(lastTaskClusterAttempt
+                            .getStatus() == TaskClusterAttempt.TaskClusterStatus.COMPLETED
+                            || lastTaskClusterAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.RUNNING)) {
+                        continue;
+                    }
+                    boolean abort = false;
+                    for (TaskAttempt ta : lastTaskClusterAttempt.getTaskAttempts().values()) {
+                        assert ta.getStatus() == TaskAttempt.TaskStatus.COMPLETED
+                                || ta.getStatus() == TaskAttempt.TaskStatus.RUNNING;
+                        if (deadNodes.contains(ta.getNodeId())) {
+                            ta.setStatus(TaskAttempt.TaskStatus.FAILED,
+                                    Collections.singletonList(new Exception("Node " + ta.getNodeId() + " failed")));
+                            ta.setEndTime(System.currentTimeMillis());
+                            abort = true;
                         }
-                        abortDoomedTaskClusters();
+                    }
+                    if (abort) {
+                        abortTaskCluster(lastTaskClusterAttempt, TaskClusterAttempt.TaskClusterStatus.ABORTED);
                     }
                 }
+                abortDoomedTaskClusters();
             }
             startRunnableActivityClusters();
         } catch (Exception e) {
@@ -706,40 +720,4 @@
         }
     }
 
-    private void finishJob(final JobRun run) {
-        JobId jobId = run.getJobId();
-        CCApplicationContext appCtx = ccs.getApplicationContext();
-        if (appCtx != null) {
-            try {
-                appCtx.notifyJobFinish(jobId);
-            } catch (HyracksException e) {
-                e.printStackTrace();
-            }
-        }
-        run.setStatus(run.getPendingStatus(), run.getPendingExceptions());
-        run.setEndTime(System.currentTimeMillis());
-        ccs.getActiveRunMap().remove(jobId);
-        ccs.getRunMapArchive().put(jobId, run);
-        ccs.getRunHistory().put(jobId, run.getExceptions());
-
-        if (run.getActivityClusterGraph().isReportTaskDetails()) {
-            /**
-             * log job details when task-profiling is enabled
-             */
-            try {
-                ccs.getJobLogFile().log(createJobLogObject(run));
-            } catch (Exception e) {
-                throw new RuntimeException(e);
-            }
-        }
-    }
-
-    private ObjectNode createJobLogObject(final JobRun run) {
-        ObjectMapper om = new ObjectMapper();
-        ObjectNode jobLogObject = om.createObjectNode();
-        ActivityClusterGraph acg = run.getActivityClusterGraph();
-        jobLogObject.set("activity-cluster-graph", acg.toJSON());
-        jobLogObject.set("job-run", run.toJSON());
-        return jobLogObject;
-    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/PartitionConstraintSolver.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/PartitionConstraintSolver.java
similarity index 96%
rename from hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/PartitionConstraintSolver.java
rename to hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/PartitionConstraintSolver.java
index 6a41d01..baafb34 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/PartitionConstraintSolver.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/PartitionConstraintSolver.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.control.cc.scheduler;
+package org.apache.hyracks.control.cc.executor;
 
 import java.util.Collection;
 import java.util.HashMap;
@@ -45,14 +45,14 @@
     public void addConstraint(Constraint c) {
         Set<ConstraintExpression> rValues = constraints.get(c.getLValue());
         if (rValues == null) {
-            rValues = new HashSet<ConstraintExpression>();
+            rValues = new HashSet<>();
             constraints.put(c.getLValue(), rValues);
         }
         rValues.add(c.getRValue());
     }
 
     public void solve(Collection<LValueConstraintExpression> targetSet) {
-        Set<LValueConstraintExpression> inProcess = new HashSet<LValueConstraintExpression>();
+        Set<LValueConstraintExpression> inProcess = new HashSet<>();
         for (LValueConstraintExpression lv : targetSet) {
             solveLValue(lv, inProcess);
         }
@@ -62,12 +62,12 @@
         switch (ce.getTag()) {
             case CONSTANT:
                 return new Solution(((ConstantExpression) ce).getValue(), Solution.Status.FOUND);
-
             case PARTITION_COUNT:
             case PARTITION_LOCATION:
                 return solveLValue((LValueConstraintExpression) ce, inProcess);
+            default:
+                return null;
         }
-        return null;
     }
 
     private Solution solveLValue(LValueConstraintExpression lv, Set<LValueConstraintExpression> inProcess) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/RankedRunnableTaskCluster.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/RankedRunnableTaskCluster.java
similarity index 80%
rename from hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/RankedRunnableTaskCluster.java
rename to hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/RankedRunnableTaskCluster.java
index a79bf50..a300c0b 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/RankedRunnableTaskCluster.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/RankedRunnableTaskCluster.java
@@ -16,13 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.control.cc.scheduler;
+package org.apache.hyracks.control.cc.executor;
 
 import org.apache.hyracks.control.cc.job.TaskCluster;
 
 public class RankedRunnableTaskCluster implements Comparable<RankedRunnableTaskCluster> {
     private final int rank;
-
     private final TaskCluster taskCluster;
 
     public RankedRunnableTaskCluster(int rank, TaskCluster taskCluster) {
@@ -48,4 +47,18 @@
         int cmp = rank - o.rank;
         return cmp < 0 ? -1 : (cmp > 0 ? 1 : 0);
     }
+
+    @Override
+    public int hashCode() {
+        return rank;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof RankedRunnableTaskCluster)) {
+            return false;
+        }
+        RankedRunnableTaskCluster target = (RankedRunnableTaskCluster) o;
+        return rank == target.rank;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/Runnability.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/Runnability.java
similarity index 98%
rename from hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/Runnability.java
rename to hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/Runnability.java
index 70d3f16..6d4b729 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/Runnability.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/executor/Runnability.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.control.cc.scheduler;
+package org.apache.hyracks.control.cc.executor;
 
 public final class Runnability {
     private final Tag tag;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/ActivityPlan.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/ActivityPlan.java
index d4df923..86a8df2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/ActivityPlan.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/ActivityPlan.java
@@ -18,7 +18,7 @@
  */
 package org.apache.hyracks.control.cc.job;
 
-import org.apache.hyracks.control.cc.scheduler.ActivityPartitionDetails;
+import org.apache.hyracks.control.cc.executor.ActivityPartitionDetails;
 
 public class ActivityPlan {
     private final ActivityPartitionDetails apd;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/IJobManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/IJobManager.java
new file mode 100644
index 0000000..8f621df
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/IJobManager.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.control.cc.job;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobStatus;
+
+/**
+ * This interface abstracts the job lifecycle management and job scheduling for a cluster.
+ */
+public interface IJobManager {
+
+    /**
+     * Enters a new job into the job manager.
+     * It's up to the implementation to decide whether to execute the job immediately, queue it for the
+     * later execution, or reject it immediately..
+     *
+     * @param jobRun,
+     *            the descriptor of a job.
+     * @throws HyracksException
+     *             when the job cannot even be accepted by the job manager, for example, when the pending
+     *             job queue is too large or the capacity requirement exceeds the capacity of the cluster.
+     */
+    void add(JobRun jobRun) throws HyracksException;
+
+    /**
+     * This method is called when the master process decides to complete job.
+     * The implementation of this method should instruct all involved worker processes to clean the state of each
+     * individual parallel partition up.
+     *
+     * If there is no involved worker processes, the method is responsible to call
+     * <code>finalComplete</code> directly, for example, when all worker processes died during the job execution.
+     *
+     * @param jobRun,
+     *            the descriptor of a job.
+     * @param status,
+     *            the final status of the job.
+     * @param exceptions,
+     *            a list of exceptions that are caught during job execution.
+     * @throws HyracksException
+     *             if anything goes wrong during the final job completion. No partial job states should be left.
+     */
+    void prepareComplete(JobRun jobRun, JobStatus status, List<Exception> exceptions) throws HyracksException;
+
+    /**
+     * This method gets called when all worker processes have notified the master that their individual parallel
+     * partition is completed and their corresponding states are cleaned up.
+     * The implementation of this method only needs to cleanup the states of a job within the master
+     * process.
+     *
+     * @param jobRun,
+     *            the descriptor of a job.
+     * @throws HyracksException
+     *             if anything goes wrong during the final job completion. No partial job states should be left.
+     */
+    void finalComplete(JobRun jobRun) throws HyracksException;
+
+    /**
+     * Retrieves a job from a given job id.
+     *
+     * @param jobId,
+     *            the id of the job.
+     * @return the job run, which is the descriptor a the job, or null if the job cannot be found.
+     */
+    JobRun get(JobId jobId);
+
+    /**
+     * Retrieves a historical job from a given job id.
+     *
+     * @param jobId,
+     *            the job id.
+     * @return the matched historical jobs that have been run but not yet discarded.
+     */
+    List<Exception> getRunHistory(JobId jobId);
+
+    /**
+     * @return all jobs that are currently running.
+     */
+    Collection<JobRun> getRunningJobs();
+
+    /**
+     * @return all jobs that are currently waiting in the job queue.
+     */
+    Collection<JobRun> getPendingJobs();
+
+    /**
+     * @return all jobs that are completed or terminated, but not yet discarded.
+     */
+    Collection<JobRun> getArchivedJobs();
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
new file mode 100644
index 0000000..52ad301
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobManager.java
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.control.cc.job;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.ActivityClusterGraph;
+import org.apache.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
+import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.application.CCApplicationContext;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
+import org.apache.hyracks.control.cc.scheduler.IJobQueue;
+import org.apache.hyracks.control.cc.work.JobCleanupWork;
+import org.apache.hyracks.control.common.controllers.CCConfig;
+import org.apache.hyracks.control.common.work.IResultCallback;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+// Job manager manages all jobs that haven been submitted to the cluster.
+public class JobManager implements IJobManager {
+
+    private static final Logger LOGGER = Logger.getLogger(JobManager.class.getName());
+
+    private final ClusterControllerService ccs;
+    private final IJobQueue jobQueue;
+    private final Map<JobId, JobRun> activeRunMap;
+    private final Map<JobId, JobRun> runMapArchive;
+    private final Map<JobId, List<Exception>> runMapHistory;
+    private final IJobCapacityController jobCapacityController;
+
+    public JobManager(CCConfig ccConfig, ClusterControllerService ccs, IJobCapacityController jobCapacityController)
+            throws HyracksException {
+        this.ccs = ccs;
+        this.jobCapacityController = jobCapacityController;
+        try {
+            Constructor<?> jobQueueConstructor = this.getClass().getClassLoader().loadClass(ccConfig.jobQueueClassName)
+                    .getConstructor(IJobManager.class, IJobCapacityController.class);
+            jobQueue = (IJobQueue) jobQueueConstructor.newInstance(this, this.jobCapacityController);
+        } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | NoSuchMethodException
+                | InvocationTargetException e) {
+            throw HyracksException.create(ErrorCode.CLASS_LOADING_ISSUE, e, e.getMessage());
+        }
+        activeRunMap = new HashMap<>();
+        runMapArchive = new LinkedHashMap<JobId, JobRun>() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<JobId, JobRun> eldest) {
+                return size() > ccConfig.jobHistorySize;
+            }
+        };
+        runMapHistory = new LinkedHashMap<JobId, List<Exception>>() {
+            private static final long serialVersionUID = 1L;
+            /** history size + 1 is for the case when history size = 0 */
+            private int allowedSize = 100 * (ccConfig.jobHistorySize + 1);
+
+            @Override
+            protected boolean removeEldestEntry(Map.Entry<JobId, List<Exception>> eldest) {
+                return size() > allowedSize;
+            }
+        };
+    }
+
+    @Override
+    public void add(JobRun jobRun) throws HyracksException {
+        checkJob(jobRun);
+        JobSpecification job = jobRun.getActivityClusterGraphFactory().getJobSpecification();
+        IJobCapacityController.JobSubmissionStatus status = jobCapacityController.allocate(job);
+        switch (status) {
+            case QUEUE:
+                jobRun.setStatus(JobStatus.PENDING, null);
+                jobQueue.add(jobRun);
+                break;
+            case EXECUTE:
+                executeJob(jobRun);
+                break;
+        }
+    }
+
+    @Override
+    public void prepareComplete(JobRun run, JobStatus status, List<Exception> exceptions) throws HyracksException {
+        checkJob(run);
+        if (status == JobStatus.FAILURE_BEFORE_EXECUTION) {
+            run.setPendingStatus(JobStatus.FAILURE, exceptions);
+            finalComplete(run);
+            return;
+        }
+        JobId jobId = run.getJobId();
+        HyracksException caughtException = null;
+        if (run.getPendingStatus() != null && run.getCleanupPendingNodeIds().isEmpty()) {
+            finalComplete(run);
+            return;
+        }
+        if (run.getPendingStatus() != null) {
+            LOGGER.warning("Ignoring duplicate cleanup for JobRun with id: " + jobId);
+            return;
+        }
+        Set<String> targetNodes = run.getParticipatingNodeIds();
+        run.getCleanupPendingNodeIds().addAll(targetNodes);
+        if (run.getPendingStatus() != JobStatus.FAILURE && run.getPendingStatus() != JobStatus.TERMINATED) {
+            run.setPendingStatus(status, exceptions);
+        }
+
+        if (targetNodes != null && !targetNodes.isEmpty()) {
+            INodeManager nodeManager = ccs.getNodeManager();
+            Set<String> toDelete = new HashSet<>();
+            for (String n : targetNodes) {
+                NodeControllerState ncs = nodeManager.getNodeControllerState(n);
+                try {
+                    if (ncs == null) {
+                        toDelete.add(n);
+                    } else {
+                        ncs.getNodeController().cleanUpJoblet(jobId, status);
+                    }
+                } catch (Exception e) {
+                    LOGGER.log(Level.SEVERE, e.getMessage(), e);
+                    if (caughtException == null) {
+                        caughtException = new HyracksException(e);
+                    } else {
+                        caughtException.addSuppressed(e);
+                    }
+                }
+            }
+            targetNodes.removeAll(toDelete);
+            run.getCleanupPendingNodeIds().removeAll(toDelete);
+            if (run.getCleanupPendingNodeIds().isEmpty()) {
+                finalComplete(run);
+            }
+        } else {
+            finalComplete(run);
+        }
+
+        // throws caught exceptions if any
+        if (caughtException != null) {
+            throw caughtException;
+        }
+    }
+
+    @Override
+    public void finalComplete(JobRun run) throws HyracksException {
+        checkJob(run);
+        JobId jobId = run.getJobId();
+        HyracksException caughtException = null;
+        CCApplicationContext appCtx = ccs.getApplicationContext();
+        if (appCtx != null) {
+            try {
+                appCtx.notifyJobFinish(jobId);
+            } catch (HyracksException e) {
+                LOGGER.log(Level.SEVERE, e.getMessage(), e);
+                caughtException = e;
+            }
+        }
+        run.setStatus(run.getPendingStatus(), run.getPendingExceptions());
+        run.setEndTime(System.currentTimeMillis());
+        activeRunMap.remove(jobId);
+        runMapArchive.put(jobId, run);
+        runMapHistory.put(jobId, run.getExceptions());
+
+        if (run.getActivityClusterGraph().isReportTaskDetails()) {
+            /**
+             * log job details when profiling is enabled
+             */
+            try {
+                ccs.getJobLogFile().log(createJobLogObject(run));
+            } catch (Exception e) {
+                LOGGER.log(Level.SEVERE, e.getMessage(), e);
+                if (caughtException == null) {
+                    caughtException = new HyracksException(e);
+                } else {
+                    caughtException.addSuppressed(e);
+                }
+            }
+        }
+
+        // Releases cluster capacitys occupied by the job.
+        JobSpecification job = run.getActivityClusterGraphFactory().getJobSpecification();
+        jobCapacityController.release(job);
+
+        // Picks the next job to execute.
+        pickJobsToRun();
+
+        // throws caught exceptions if any
+        if (caughtException != null) {
+            throw caughtException;
+        }
+    }
+
+
+
+    @Override
+    public Collection<JobRun> getRunningJobs() {
+        return activeRunMap.values();
+    }
+
+    @Override
+    public Collection<JobRun> getPendingJobs() {
+        return jobQueue.jobs();
+    }
+
+    @Override
+    public Collection<JobRun> getArchivedJobs() {
+        return runMapArchive.values();
+    }
+
+    @Override
+    public JobRun get(JobId jobId) {
+        JobRun jobRun = activeRunMap.get(jobId);
+        if (jobRun == null) {
+            jobRun = runMapArchive.get(jobId);
+        }
+        return jobRun;
+    }
+
+    @Override
+    public List<Exception> getRunHistory(JobId jobId) {
+        return runMapHistory.get(jobId);
+    }
+
+    private void pickJobsToRun() {
+        List<JobRun> selectedRuns = jobQueue.pull();
+        for (JobRun run : selectedRuns) {
+            executeJob(run);
+        }
+    }
+
+    // Executes a job when the required capacity for the job is met.
+    private void executeJob(JobRun run) {
+        IResultCallback<JobId> callback = run.getCallback();
+        try {
+            run.setStartTime(System.currentTimeMillis());
+            JobId jobId = run.getJobId();
+            activeRunMap.put(jobId, run);
+
+            CCApplicationContext appCtx = ccs.getApplicationContext();
+            IActivityClusterGraphGeneratorFactory acggf = run.getActivityClusterGraphFactory();
+            appCtx.notifyJobCreation(jobId, acggf);
+            run.setStatus(JobStatus.RUNNING, null);
+            executeJobInternal(run);
+            callback.setValue(jobId);
+        } catch (Exception e) {
+            callback.setException(e);
+        }
+    }
+
+    private void executeJobInternal(JobRun run) {
+        try {
+            run.getExecutor().startJob();
+        } catch (Exception e) {
+            ccs.getWorkQueue()
+                    .schedule(new JobCleanupWork(ccs.getJobManager(), run.getJobId(), JobStatus.FAILURE,
+                            Collections.singletonList(e)));
+        }
+    }
+
+    private ObjectNode createJobLogObject(final JobRun run) {
+        ObjectMapper om = new ObjectMapper();
+        ObjectNode jobLogObject = om.createObjectNode();
+        ActivityClusterGraph acg = run.getActivityClusterGraph();
+        jobLogObject.set("activity-cluster-graph", acg.toJSON());
+        jobLogObject.set("job-run", run.toJSON());
+        return jobLogObject;
+    }
+
+    private void checkJob(JobRun jobRun) throws HyracksException {
+        if (jobRun == null) {
+            throw HyracksException.create(ErrorCode.INVALID_INPUT_PARAMETER);
+        }
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
index f1d04bb..5682194 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
@@ -20,17 +20,12 @@
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.hyracks.api.dataflow.ActivityId;
 import org.apache.hyracks.api.dataflow.ConnectorDescriptorId;
 import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
@@ -42,29 +37,37 @@
 import org.apache.hyracks.api.job.ActivityClusterGraph;
 import org.apache.hyracks.api.job.ActivityClusterId;
 import org.apache.hyracks.api.job.IActivityClusterGraphGenerator;
+import org.apache.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.api.partitions.PartitionId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.executor.ActivityPartitionDetails;
+import org.apache.hyracks.control.cc.executor.JobExecutor;
 import org.apache.hyracks.control.cc.partitions.PartitionMatchMaker;
-import org.apache.hyracks.control.cc.scheduler.ActivityPartitionDetails;
-import org.apache.hyracks.control.cc.scheduler.JobScheduler;
 import org.apache.hyracks.control.common.job.profiling.om.JobProfile;
 import org.apache.hyracks.control.common.utils.ExceptionUtils;
+import org.apache.hyracks.control.common.work.IResultCallback;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 public class JobRun implements IJobStatusConditionVariable {
     private final DeploymentId deploymentId;
 
     private final JobId jobId;
 
+    private final IActivityClusterGraphGeneratorFactory acggf;
+
     private final IActivityClusterGraphGenerator acgg;
 
     private final ActivityClusterGraph acg;
 
-    private final JobScheduler scheduler;
+    private final JobExecutor scheduler;
 
-    private final EnumSet<JobFlag> jobFlags;
+    private final Set<JobFlag> jobFlags;
 
     private final Map<ActivityClusterId, ActivityClusterPlan> activityClusterPlanMap;
 
@@ -94,21 +97,26 @@
 
     private Map<OperatorDescriptorId, Map<Integer, String>> operatorLocations;
 
+    private final IResultCallback<JobId> callback;
+
     public JobRun(ClusterControllerService ccs, DeploymentId deploymentId, JobId jobId,
-            IActivityClusterGraphGenerator acgg, EnumSet<JobFlag> jobFlags) {
+            IActivityClusterGraphGeneratorFactory acggf, IActivityClusterGraphGenerator acgg, Set<JobFlag> jobFlags,
+            IResultCallback<JobId> callback) {
         this.deploymentId = deploymentId;
         this.jobId = jobId;
+        this.acggf = acggf;
         this.acgg = acgg;
         this.acg = acgg.initialize();
-        this.scheduler = new JobScheduler(ccs, this, acgg.getConstraints());
+        this.scheduler = new JobExecutor(ccs, this, acgg.getConstraints());
         this.jobFlags = jobFlags;
-        activityClusterPlanMap = new HashMap<ActivityClusterId, ActivityClusterPlan>();
+        this.callback = callback;
+        activityClusterPlanMap = new HashMap<>();
         pmm = new PartitionMatchMaker();
-        participatingNodeIds = new HashSet<String>();
-        cleanupPendingNodeIds = new HashSet<String>();
+        participatingNodeIds = new HashSet<>();
+        cleanupPendingNodeIds = new HashSet<>();
         profile = new JobProfile(jobId);
-        connectorPolicyMap = new HashMap<ConnectorDescriptorId, IConnectorPolicy>();
-        operatorLocations = new HashMap<OperatorDescriptorId, Map<Integer, String>>();
+        connectorPolicyMap = new HashMap<>();
+        operatorLocations = new HashMap<>();
         createTime = System.currentTimeMillis();
     }
 
@@ -120,11 +128,15 @@
         return jobId;
     }
 
+    public IActivityClusterGraphGeneratorFactory getActivityClusterGraphFactory() {
+        return acggf;
+    }
+
     public ActivityClusterGraph getActivityClusterGraph() {
         return acg;
     }
 
-    public EnumSet<JobFlag> getFlags() {
+    public Set<JobFlag> getFlags() {
         return jobFlags;
     }
 
@@ -167,8 +179,8 @@
         return createTime;
     }
 
-    public void setCreateTime(long createTime) {
-        this.createTime = createTime;
+    public IResultCallback<JobId> getCallback() {
+        return callback;
     }
 
     public long getStartTime() {
@@ -228,7 +240,7 @@
         return profile;
     }
 
-    public JobScheduler getScheduler() {
+    public JobExecutor getExecutor() {
         return scheduler;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java
index cca56fc..3a5e3be 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionMatchMaker.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.control.cc.partitions;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -27,7 +28,6 @@
 import java.util.logging.Logger;
 
 import org.apache.commons.lang3.tuple.Pair;
-
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.partitions.PartitionId;
 import org.apache.hyracks.control.common.job.PartitionDescriptor;
@@ -156,7 +156,7 @@
         }
     }
 
-    public void notifyNodeFailures(final Set<String> deadNodes) {
+    public void notifyNodeFailures(final Collection<String> deadNodes) {
         removeEntries(partitionDescriptors, new IEntryFilter<PartitionDescriptor>() {
             @Override
             public boolean matches(PartitionDescriptor o) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionUtils.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionUtils.java
index 807798e..65851ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/partitions/PartitionUtils.java
@@ -16,14 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.control.cc.partitions;
 
 import org.apache.commons.lang3.tuple.Pair;
-
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.partitions.PartitionId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.base.INodeController;
 import org.apache.hyracks.control.common.job.PartitionDescriptor;
 import org.apache.hyracks.control.common.job.PartitionRequest;
@@ -34,8 +35,9 @@
         PartitionDescriptor desc = match.getLeft();
         PartitionRequest req = match.getRight();
 
-        NodeControllerState producerNCS = ccs.getNodeMap().get(desc.getNodeId());
-        NodeControllerState requestorNCS = ccs.getNodeMap().get(req.getNodeId());
+        INodeManager nodeManager = ccs.getNodeManager();
+        NodeControllerState producerNCS = nodeManager.getNodeControllerState(desc.getNodeId());
+        NodeControllerState requestorNCS = nodeManager.getNodeControllerState(req.getNodeId());
         final NetworkAddress dataport = producerNCS.getDataPort();
         final INodeController requestorNC = requestorNCS.getNodeController();
         requestorNC.reportPartitionAvailability(pid, dataport);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/FIFOJobQueue.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/FIFOJobQueue.java
new file mode 100644
index 0000000..eac9800
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/FIFOJobQueue.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.control.cc.scheduler;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
+import org.apache.hyracks.control.cc.job.IJobManager;
+import org.apache.hyracks.control.cc.job.JobRun;
+
+/**
+ * An implementation of IJobQueue that gives more priority to jobs that are submitted earlier.
+ */
+public class FIFOJobQueue implements IJobQueue {
+
+    private static final Logger LOGGER = Logger.getLogger(FIFOJobQueue.class.getName());
+
+    private static final int CAPACITY = 4096;
+    private final List<JobRun> jobQueue = new LinkedList<>();
+    private final IJobManager jobManager;
+    private final IJobCapacityController jobCapacityController;
+
+    public FIFOJobQueue(IJobManager jobManager, IJobCapacityController jobCapacityController) {
+        this.jobManager = jobManager;
+        this.jobCapacityController = jobCapacityController;
+    }
+
+    @Override
+    public void add(JobRun run) throws HyracksException {
+        int size = jobQueue.size();
+        if (size >= CAPACITY) {
+            throw HyracksException.create(ErrorCode.JOB_QUEUE_FULL, new Integer(CAPACITY));
+        }
+        jobQueue.add(run);
+    }
+
+    @Override
+    public List<JobRun> pull() {
+        List<JobRun> jobRuns = new ArrayList<>();
+        Iterator<JobRun> runIterator = jobQueue.iterator();
+        while (runIterator.hasNext()) {
+            JobRun run = runIterator.next();
+            JobSpecification job = run.getActivityClusterGraphFactory().getJobSpecification();
+            // Cluster maximum capacity can change over time, thus we have to re-check if the job should be rejected
+            // or not.
+            try {
+                IJobCapacityController.JobSubmissionStatus status = jobCapacityController.allocate(job);
+                // Checks if the job can be executed immediately.
+                if (status == IJobCapacityController.JobSubmissionStatus.EXECUTE) {
+                    jobRuns.add(run);
+                    runIterator.remove(); // Removes the selected job.
+                }
+            } catch (HyracksException exception) {
+                // The required capacity exceeds maximum capacity.
+                List<Exception> exceptions = new ArrayList<>();
+                exceptions.add(exception);
+                runIterator.remove(); // Removes the job from the queue.
+                try {
+                    // Fails the job.
+                    jobManager.prepareComplete(run, JobStatus.FAILURE_BEFORE_EXECUTION, exceptions);
+                } catch (HyracksException e) {
+                    LOGGER.log(Level.SEVERE, e.getMessage(), e);
+                }
+                continue;
+            }
+        }
+        return jobRuns;
+    }
+
+    @Override
+    public Collection<JobRun> jobs() {
+        return Collections.unmodifiableCollection(jobQueue);
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/IJobQueue.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/IJobQueue.java
new file mode 100644
index 0000000..2c26799
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/IJobQueue.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.control.cc.scheduler;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.control.cc.job.JobRun;
+
+/**
+ * This interface specifies a job queue.
+ */
+public interface IJobQueue {
+
+    /**
+     * Adds a job into the job queue.
+     *
+     * @param run,
+     *            the descriptor of a job.
+     * @throws HyracksException
+     *             when the size of the queue exceeds its capacity.
+     */
+    void add(JobRun run) throws HyracksException;
+
+    /**
+     * Pull a list of jobs from the job queque, when more cluster capacity becomes available.
+     *
+     * @return a list of jobs whose capacity requirements can all be met at the same time.
+     */
+    List<JobRun> pull();
+
+    /**
+     * @return all pending jobs in the queue.
+     */
+    Collection<JobRun> jobs();
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/IResourceManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/IResourceManager.java
new file mode 100644
index 0000000..a4ac9e7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/IResourceManager.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.control.cc.scheduler;
+
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.resource.IClusterCapacity;
+import org.apache.hyracks.api.job.resource.IReadOnlyClusterCapacity;
+import org.apache.hyracks.api.job.resource.NodeCapacity;
+
+/**
+ * This interface abstracts the resource management of a cluster.
+ */
+public interface IResourceManager {
+
+    /**
+     * @return the maximum capacity of the cluster, assuming that there is no running job
+     *         that occupies capacity.
+     */
+    IReadOnlyClusterCapacity getMaximumCapacity();
+
+    /**
+     * @return the current capacity for computation.
+     */
+    IClusterCapacity getCurrentCapacity();
+
+    /**
+     * Updates the cluster capacity when a node is added, removed, or updated.
+     *
+     * @param nodeId,
+     *            the id of the node for updating.
+     * @param capacity,
+     *            the capacity of one particular node.
+     * @throws HyracksException
+     *             when the parameters are invalid.
+     */
+    void update(String nodeId, NodeCapacity capacity) throws HyracksException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/ResourceManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/ResourceManager.java
new file mode 100644
index 0000000..6168dce
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/scheduler/ResourceManager.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.control.cc.scheduler;
+
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.resource.ClusterCapacity;
+import org.apache.hyracks.api.job.resource.IClusterCapacity;
+import org.apache.hyracks.api.job.resource.IReadOnlyClusterCapacity;
+import org.apache.hyracks.api.job.resource.NodeCapacity;
+
+public class ResourceManager implements IResourceManager {
+
+    // The maximum capacity, assuming that there is no running job that occupies capacity.
+    // It is unchanged unless any node is added, removed or updated.
+    private IClusterCapacity maxCapacity = new ClusterCapacity();
+
+    // The current capacity, which is dynamically changing.
+    private IClusterCapacity currentCapacity = new ClusterCapacity();
+
+    @Override
+    public IReadOnlyClusterCapacity getMaximumCapacity() {
+        return maxCapacity;
+    }
+
+    @Override
+    public IClusterCapacity getCurrentCapacity() {
+        return currentCapacity;
+    }
+
+    @Override
+    public void update(String nodeId, NodeCapacity capacity) throws HyracksException {
+        maxCapacity.update(nodeId, capacity);
+        currentCapacity.update(nodeId, capacity);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/JobsRESTAPIFunction.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/JobsRESTAPIFunction.java
index e69884a..892807b 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/JobsRESTAPIFunction.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/JobsRESTAPIFunction.java
@@ -45,7 +45,7 @@
                     break;
                 }
             case 0: {
-                GetJobSummariesJSONWork gjse = new GetJobSummariesJSONWork(ccs);
+                GetJobSummariesJSONWork gjse = new GetJobSummariesJSONWork(ccs.getJobManager());
                 ccs.getWorkQueue().scheduleAndSync(gjse);
                 result.set("result", gjse.getSummaries());
                 break;
@@ -59,7 +59,7 @@
                     ccs.getWorkQueue().scheduleAndSync(gjage);
                     result.set("result", gjage.getJSON());
                 } else if ("job-run".equalsIgnoreCase(arguments[1])) {
-                    GetJobRunJSONWork gjre = new GetJobRunJSONWork(ccs, jobId);
+                    GetJobRunJSONWork gjre = new GetJobRunJSONWork(ccs.getJobManager(), jobId);
                     ccs.getWorkQueue().scheduleAndSync(gjre);
                     result.set("result", gjre.getJSON());
                 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/NodesRESTAPIFunction.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/NodesRESTAPIFunction.java
index 8994895..3b4918c 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/NodesRESTAPIFunction.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/NodesRESTAPIFunction.java
@@ -18,14 +18,14 @@
  */
 package org.apache.hyracks.control.cc.web;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.web.util.IJSONOutputFunction;
 import org.apache.hyracks.control.cc.work.GetNodeDetailsJSONWork;
 import org.apache.hyracks.control.cc.work.GetNodeSummariesJSONWork;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class NodesRESTAPIFunction implements IJSONOutputFunction {
     private ClusterControllerService ccs;
 
@@ -40,12 +40,13 @@
         switch (arguments.length) {
             case 1: {
                 if ("".equals(arguments[0])) {
-                    GetNodeSummariesJSONWork gnse = new GetNodeSummariesJSONWork(ccs);
+                    GetNodeSummariesJSONWork gnse = new GetNodeSummariesJSONWork(ccs.getNodeManager());
                     ccs.getWorkQueue().scheduleAndSync(gnse);
                     result.set("result", gnse.getSummaries());
                 } else {
                     String nodeId = arguments[0];
-                    GetNodeDetailsJSONWork gnde = new GetNodeDetailsJSONWork(ccs, nodeId, true, true);
+                    GetNodeDetailsJSONWork gnde = new GetNodeDetailsJSONWork(ccs.getNodeManager(), ccs.getCCConfig(),
+                            nodeId, true, true);
                     ccs.getWorkQueue().scheduleAndSync(gnde);
                     result.set("result", gnde.getDetail());
                 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/WebServer.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/WebServer.java
index b55e65d..8fe68c1 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/WebServer.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/WebServer.java
@@ -19,7 +19,6 @@
 package org.apache.hyracks.control.cc.web;
 
 import java.util.EnumSet;
-import java.util.logging.Logger;
 
 import javax.servlet.DispatcherType;
 
@@ -43,8 +42,6 @@
 import org.eclipse.jetty.servlet.ServletContextHandler;
 
 public class WebServer {
-    private final static Logger LOGGER = Logger.getLogger(WebServer.class.getName());
-
     private final ClusterControllerService ccs;
     private final Server server;
     private final ServerConnector connector;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractHeartbeatWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractHeartbeatWork.java
index ad333fe..fa5dcd0 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractHeartbeatWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractHeartbeatWork.java
@@ -19,10 +19,9 @@
 
 package org.apache.hyracks.control.cc.work;
 
-import java.util.Map;
-
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.heartbeat.HeartbeatData;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 
@@ -40,8 +39,8 @@
 
     @Override
     public void doRun() {
-        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
-        NodeControllerState state = nodeMap.get(nodeId);
+        INodeManager nodeManager = ccs.getNodeManager();
+        NodeControllerState state = nodeManager.getNodeControllerState(nodeId);
         if (state != null) {
             state.notifyHeartbeat(hbData);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractTaskLifecycleWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractTaskLifecycleWork.java
index 9134a91..3babf00 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractTaskLifecycleWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/AbstractTaskLifecycleWork.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.job.ActivityPlan;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.cc.job.Task;
 import org.apache.hyracks.control.cc.job.TaskAttempt;
@@ -50,7 +51,8 @@
 
     @Override
     public final void runWork() {
-        JobRun run = ccs.getActiveRunMap().get(jobId);
+        IJobManager jobManager = ccs.getJobManager();
+        JobRun run = jobManager.get(jobId);
         if (run != null) {
             TaskId tid = taId.getTaskId();
             Map<ActivityId, ActivityCluster> activityClusterMap = run.getActivityClusterGraph().getActivityMap();
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliDeployBinaryWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliDeployBinaryWork.java
index 741c641..6480674 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliDeployBinaryWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliDeployBinaryWork.java
@@ -20,15 +20,14 @@
 package org.apache.hyracks.control.cc.work;
 
 import java.net.URL;
+import java.util.Collection;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
 import java.util.UUID;
 
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.deployment.DeploymentRun;
 import org.apache.hyracks.control.common.deployment.DeploymentUtils;
 import org.apache.hyracks.control.common.work.IPCResponder;
@@ -72,12 +71,8 @@
             /**
              * Deploy for the node controllers
              */
-            Map<String, NodeControllerState> nodeControllerStateMap = ccs.getNodeMap();
-
-            Set<String> nodeIds = new TreeSet<String>();
-            for (String nc : nodeControllerStateMap.keySet()) {
-                nodeIds.add(nc);
-            }
+            INodeManager nodeManager = ccs.getNodeManager();
+            Collection<String> nodeIds = nodeManager.getAllNodeIds();
             final DeploymentRun dRun = new DeploymentRun(nodeIds);
 
             /** The following call prevents a user to deploy with the same deployment id simultaneously. */
@@ -86,7 +81,7 @@
             /***
              * deploy binaries to each node controller
              */
-            for (NodeControllerState ncs : nodeControllerStateMap.values()) {
+            for (NodeControllerState ncs : nodeManager.getAllNodeControllerStates()) {
                 ncs.getNodeController().deployBinary(deploymentId, binaryURLs);
             }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliUnDeployBinaryWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliUnDeployBinaryWork.java
index 5f97ce2..de28c32 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliUnDeployBinaryWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/CliUnDeployBinaryWork.java
@@ -19,14 +19,14 @@
 
 package org.apache.hyracks.control.cc.work;
 
-import java.util.Map;
+import java.util.Collection;
 import java.util.Set;
-import java.util.TreeSet;
 import java.util.UUID;
 
 import org.apache.hyracks.api.deployment.DeploymentId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.deployment.DeploymentRun;
 import org.apache.hyracks.control.common.deployment.DeploymentUtils;
 import org.apache.hyracks.control.common.work.IPCResponder;
@@ -68,12 +68,8 @@
             /**
              * Deploy for the node controllers
              */
-            Map<String, NodeControllerState> nodeControllerStateMap = ccs.getNodeMap();
-
-            Set<String> nodeIds = new TreeSet<String>();
-            for (String nc : nodeControllerStateMap.keySet()) {
-                nodeIds.add(nc);
-            }
+            INodeManager nodeManager = ccs.getNodeManager();
+            Collection<String> nodeIds = nodeManager.getAllNodeIds();
             final DeploymentRun dRun = new DeploymentRun(nodeIds);
 
             /** The following call prevents a user to undeploy with the same deployment id simultaneously. */
@@ -82,7 +78,7 @@
             /***
              * deploy binaries to each node controller
              */
-            for (NodeControllerState ncs : nodeControllerStateMap.values()) {
+            for (NodeControllerState ncs : nodeManager.getAllNodeControllerStates()) {
                 ncs.getNodeController().undeployBinary(deploymentId);
             }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ClusterShutdownWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ClusterShutdownWork.java
index e05dfbc..0b89f55 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ClusterShutdownWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ClusterShutdownWork.java
@@ -19,14 +19,13 @@
 
 package org.apache.hyracks.control.cc.work;
 
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
+import java.util.Collection;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.shutdown.ShutdownRun;
 import org.apache.hyracks.control.common.work.IResultCallback;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
@@ -52,9 +51,8 @@
             if (ccs.getShutdownRun() != null) {
                 throw new IPCException("Shutdown already in progress");
             }
-            Map<String, NodeControllerState> nodeControllerStateMap = ccs.getNodeMap();
-            Set<String> nodeIds = new TreeSet<>();
-            nodeIds.addAll(nodeControllerStateMap.keySet());
+            INodeManager nodeManager = ccs.getNodeManager();
+            Collection<String> nodeIds = nodeManager.getAllNodeIds();
             /**
              * set up our listener for the node ACKs
              */
@@ -64,7 +62,7 @@
             /**
              * Shutdown all the nodes...
              */
-            nodeControllerStateMap.forEach(this::shutdownNode);
+            nodeManager.apply(this::shutdownNode);
 
             ccs.getExecutor().execute(new Runnable() {
                 @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GatherStateDumpsWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GatherStateDumpsWork.java
index 91a5906..7709a2b 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GatherStateDumpsWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GatherStateDumpsWork.java
@@ -16,16 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.control.cc.work;
 
+import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
-import java.util.Set;
 import java.util.UUID;
 
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 
 public class GatherStateDumpsWork extends SynchronizableWork {
@@ -41,8 +42,9 @@
     @Override
     public void doRun() throws Exception {
         ccs.addStateDumpRun(sdr.stateDumpId, sdr);
-        sdr.setNCs(new HashSet<>(ccs.getNodeMap().keySet()));
-        for (NodeControllerState ncs : ccs.getNodeMap().values()) {
+        INodeManager nodeManager = ccs.getNodeManager();
+        sdr.setNCs(nodeManager.getAllNodeIds());
+        for (NodeControllerState ncs : nodeManager.getAllNodeControllerStates()) {
             ncs.getNodeController().dumpState(sdr.stateDumpId);
         }
     }
@@ -59,7 +61,7 @@
 
         private final Map<String, String> ncStates;
 
-        private Set<String> ncIds;
+        private Collection<String> ncIds;
 
         private boolean complete;
 
@@ -70,7 +72,7 @@
             complete = false;
         }
 
-        public void setNCs(Set<String> ncIds) {
+        public void setNCs(Collection<String> ncIds) {
             this.ncIds = ncIds;
         }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetActivityClusterGraphJSONWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetActivityClusterGraphJSONWork.java
index 294ae97..6a4a1d6 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetActivityClusterGraphJSONWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetActivityClusterGraphJSONWork.java
@@ -18,14 +18,15 @@
  */
 package org.apache.hyracks.control.cc.work;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class GetActivityClusterGraphJSONWork extends SynchronizableWork {
     private final ClusterControllerService ccs;
     private final JobId jobId;
@@ -38,15 +39,12 @@
 
     @Override
     protected void doRun() throws Exception {
-
+        IJobManager jobManager = ccs.getJobManager();
         ObjectMapper om = new ObjectMapper();
-        JobRun run = ccs.getActiveRunMap().get(jobId);
+        JobRun run = jobManager.get(jobId);
         if (run == null) {
-            run = ccs.getRunMapArchive().get(jobId);
-            if (run == null) {
-                json = om.createObjectNode();
-                return;
-            }
+            json = om.createObjectNode();
+            return;
         }
         json = run.getActivityClusterGraph().toJSON();
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java
index 31a829c..872fb9c 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetIpAddressNodeNameMapWork.java
@@ -23,19 +23,20 @@
 import java.util.Set;
 
 import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 
 public class GetIpAddressNodeNameMapWork extends SynchronizableWork {
-    private final ClusterControllerService ccs;
+    private final INodeManager nodeManager;
     private Map<InetAddress, Set<String>> map;
 
-    public GetIpAddressNodeNameMapWork(ClusterControllerService ccs, Map<InetAddress, Set<String>> map) {
-        this.ccs = ccs;
+    public GetIpAddressNodeNameMapWork(INodeManager nodeManager, Map<InetAddress, Set<String>> map) {
+        this.nodeManager = nodeManager;
         this.map = map;
     }
 
     @Override
     protected void doRun() throws Exception {
-        map.putAll(ccs.getIpAddressNodeNameMap());
+        map.putAll(nodeManager.getIpAddressNodeNameMap());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobInfoWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobInfoWork.java
index e072c21..8fe6470 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobInfoWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobInfoWork.java
@@ -20,18 +20,18 @@
 
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobInfo;
-import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.common.work.IResultCallback;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 
 public class GetJobInfoWork extends SynchronizableWork {
-    private final ClusterControllerService ccs;
+    private final IJobManager jobManager;
     private final JobId jobId;
     private final IResultCallback<JobInfo> callback;
 
-    public GetJobInfoWork(ClusterControllerService ccs, JobId jobId, IResultCallback<JobInfo> callback) {
-        this.ccs = ccs;
+    public GetJobInfoWork(IJobManager jobManager, JobId jobId, IResultCallback<JobInfo> callback) {
+        this.jobManager = jobManager;
         this.jobId = jobId;
         this.callback = callback;
     }
@@ -39,10 +39,7 @@
     @Override
     protected void doRun() throws Exception {
         try {
-            JobRun run = ccs.getActiveRunMap().get(jobId);
-            if (run == null) {
-                run = ccs.getRunMapArchive().get(jobId);
-            }
+            JobRun run = jobManager.get(jobId);
             JobInfo info = (run != null) ? new JobInfo(run.getJobId(), run.getStatus(), run.getOperatorLocations())
                     : null;
             callback.setValue(info);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobRunJSONWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobRunJSONWork.java
index aad6edf..3a7c449 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobRunJSONWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobRunJSONWork.java
@@ -18,34 +18,31 @@
  */
 package org.apache.hyracks.control.cc.work;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-
 import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class GetJobRunJSONWork extends SynchronizableWork {
-    private final ClusterControllerService ccs;
+    private final IJobManager jobManager;
     private final JobId jobId;
     private ObjectNode json;
 
-    public GetJobRunJSONWork(ClusterControllerService ccs, JobId jobId) {
-        this.ccs = ccs;
+    public GetJobRunJSONWork(IJobManager jobManager, JobId jobId) {
+        this.jobManager = jobManager;
         this.jobId = jobId;
     }
 
     @Override
     protected void doRun() throws Exception {
         ObjectMapper om = new ObjectMapper();
-        JobRun run = ccs.getActiveRunMap().get(jobId);
+        JobRun run = jobManager.get(jobId);
         if (run == null) {
-            run = ccs.getRunMapArchive().get(jobId);
-            if (run == null) {
-                json = om.createObjectNode();
-                return;
-            }
+            json = om.createObjectNode();
+            return;
         }
         json = run.toJSON();
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobStatusWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobStatusWork.java
index d45a9cc..b5bf8b2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobStatusWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobStatusWork.java
@@ -16,22 +16,23 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.control.cc.work;
 
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobStatus;
-import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.common.work.IResultCallback;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 
 public class GetJobStatusWork extends SynchronizableWork {
-    private final ClusterControllerService ccs;
+    private final IJobManager jobManager;
     private final JobId jobId;
     private final IResultCallback<JobStatus> callback;
 
-    public GetJobStatusWork(ClusterControllerService ccs, JobId jobId, IResultCallback<JobStatus> callback) {
-        this.ccs = ccs;
+    public GetJobStatusWork(IJobManager jobManager, JobId jobId, IResultCallback<JobStatus> callback) {
+        this.jobManager = jobManager;
         this.jobId = jobId;
         this.callback = callback;
     }
@@ -39,10 +40,7 @@
     @Override
     protected void doRun() throws Exception {
         try {
-            JobRun run = ccs.getActiveRunMap().get(jobId);
-            if (run == null) {
-                run = ccs.getRunMapArchive().get(jobId);
-            }
+            JobRun run = jobManager.get(jobId);
             JobStatus status = run == null ? null : run.getStatus();
             callback.setValue(status);
         } catch (Exception e) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobSummariesJSONWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobSummariesJSONWork.java
index 1e5a3a5..9c680c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobSummariesJSONWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetJobSummariesJSONWork.java
@@ -16,31 +16,34 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.control.cc.work;
 
 import java.util.Collection;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class GetJobSummariesJSONWork extends SynchronizableWork {
-    private final ClusterControllerService ccs;
+    private final IJobManager jobManager;
     private ArrayNode summaries;
 
-    public GetJobSummariesJSONWork(ClusterControllerService ccs) {
-        this.ccs = ccs;
+    public GetJobSummariesJSONWork(IJobManager jobManager) {
+        this.jobManager = jobManager;
     }
 
     @Override
     protected void doRun() throws Exception {
         ObjectMapper om = new ObjectMapper();
         summaries = om.createArrayNode();
-        populateJSON(ccs.getActiveRunMap().values());
-        populateJSON(ccs.getRunMapArchive().values());
+        populateJSON(jobManager.getRunningJobs());
+        populateJSON(jobManager.getPendingJobs());
+        populateJSON(jobManager.getArchivedJobs());
     }
 
     private void populateJSON(Collection<JobRun> jobRuns)  {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
index a0150f2..c36b887 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
@@ -18,35 +18,25 @@
  */
 package org.apache.hyracks.control.cc.work;
 
-import java.util.LinkedHashMap;
 import java.util.Map;
 
 import org.apache.hyracks.api.client.NodeControllerInfo;
-import org.apache.hyracks.api.client.NodeStatus;
-import org.apache.hyracks.control.cc.ClusterControllerService;
-import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.work.AbstractWork;
 import org.apache.hyracks.control.common.work.IResultCallback;
 
 public class GetNodeControllersInfoWork extends AbstractWork {
-    private final ClusterControllerService ccs;
+    private final INodeManager nodeManager;
     private IResultCallback<Map<String, NodeControllerInfo>> callback;
 
-    public GetNodeControllersInfoWork(ClusterControllerService ccs,
+    public GetNodeControllersInfoWork(INodeManager nodeManager,
             IResultCallback<Map<String, NodeControllerInfo>> callback) {
-        this.ccs = ccs;
+        this.nodeManager = nodeManager;
         this.callback = callback;
     }
 
     @Override
     public void run() {
-        Map<String, NodeControllerInfo> result = new LinkedHashMap<>();
-        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
-        for (Map.Entry<String, NodeControllerState> e : nodeMap.entrySet()) {
-            NodeControllerState ncState = e.getValue();
-            result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE, ncState.getDataPort(),
-                    ncState.getDatasetPort(), ncState.getMessagingPort(), ncState.getNumCores()));
-        }
-        callback.setValue(result);
+        callback.setValue(nodeManager.getNodeControllerInfoMap());
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
index 4a8dd1f..0577002 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeDetailsJSONWork.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.control.cc.work;
 
 import java.io.File;
@@ -35,18 +36,21 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
+import org.apache.hyracks.control.common.controllers.CCConfig;
 import org.apache.hyracks.control.common.utils.PidHelper;
 import org.apache.hyracks.control.common.work.IPCResponder;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 import org.kohsuke.args4j.Option;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
 public class GetNodeDetailsJSONWork extends SynchronizableWork {
     private static final Logger LOGGER = Logger.getLogger(GetNodeDetailsJSONWork.class.getName());
-    private final ClusterControllerService ccs;
+    private final INodeManager nodeManager;
+    private final CCConfig ccConfig;
     private final String nodeId;
     private final boolean includeStats;
     private final boolean includeConfig;
@@ -54,18 +58,19 @@
     private ObjectNode detail;
     private ObjectMapper om = new ObjectMapper();
 
-    public GetNodeDetailsJSONWork(ClusterControllerService ccs, String nodeId, boolean includeStats,
+    public GetNodeDetailsJSONWork(INodeManager nodeManager, CCConfig ccConfig, String nodeId, boolean includeStats,
                                   boolean includeConfig, IPCResponder<String> callback) {
-        this.ccs = ccs;
+        this.nodeManager = nodeManager;
+        this.ccConfig = ccConfig;
         this.nodeId = nodeId;
         this.includeStats = includeStats;
         this.includeConfig = includeConfig;
         this.callback = callback;
     }
 
-    public GetNodeDetailsJSONWork(ClusterControllerService ccs, String nodeId, boolean includeStats,
+    public GetNodeDetailsJSONWork(INodeManager nodeManager, CCConfig ccConfig, String nodeId, boolean includeStats,
                                   boolean includeConfig) {
-        this(ccs, nodeId, includeStats, includeConfig, null);
+        this(nodeManager, ccConfig, nodeId, includeStats, includeConfig, null);
     }
 
     @Override
@@ -74,10 +79,10 @@
             // null nodeId is a request for CC
             detail = getCCDetails();
             if (includeConfig) {
-                addIni(detail, ccs.getCCConfig());
+                addIni(detail, ccConfig);
             }
         } else {
-            NodeControllerState ncs = ccs.getNodeMap().get(nodeId);
+            NodeControllerState ncs = nodeManager.getNodeControllerState(nodeId);
             if (ncs != null) {
                 detail = ncs.toDetailedJSON(includeStats, includeConfig);
                 if (includeConfig) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeSummariesJSONWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeSummariesJSONWork.java
index ebafd7d..b78e817 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeSummariesJSONWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeSummariesJSONWork.java
@@ -18,26 +18,26 @@
  */
 package org.apache.hyracks.control.cc.work;
 
+import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
+import org.apache.hyracks.control.common.work.SynchronizableWork;
+
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;
 
-import org.apache.hyracks.control.cc.ClusterControllerService;
-import org.apache.hyracks.control.cc.NodeControllerState;
-import org.apache.hyracks.control.common.work.SynchronizableWork;
-
 public class GetNodeSummariesJSONWork extends SynchronizableWork {
-    private final ClusterControllerService ccs;
+    private final INodeManager nodeManager;
     private ArrayNode summaries;
 
-    public GetNodeSummariesJSONWork(ClusterControllerService ccs) {
-        this.ccs = ccs;
+    public GetNodeSummariesJSONWork(INodeManager nodeManager) {
+        this.nodeManager = nodeManager;
     }
 
     @Override
     protected void doRun() throws Exception {
         ObjectMapper om = new ObjectMapper();
         summaries = om.createArrayNode();
-        for (NodeControllerState ncs : ccs.getNodeMap().values()) {
+        for (NodeControllerState ncs : nodeManager.getAllNodeControllerStates()) {
             summaries.add(ncs.toSummaryJSON());
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetThreadDumpWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetThreadDumpWork.java
index 889c828..d7c7be1 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetThreadDumpWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetThreadDumpWork.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.control.cc.work;
 
 import java.lang.management.ManagementFactory;
@@ -27,6 +28,7 @@
 
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.utils.ThreadDumpHelper;
 import org.apache.hyracks.control.common.work.AbstractWork;
 import org.apache.hyracks.control.common.work.IResultCallback;
@@ -59,7 +61,8 @@
                 callback.setException(e);
             }
         } else {
-            final NodeControllerState ncState = ccs.getNodeMap().get(nodeId);
+            INodeManager nodeManager = ccs.getNodeManager();
+            final NodeControllerState ncState = nodeManager.getNodeControllerState(nodeId);
             if (ncState == null) {
                 // bad node id, reply with null immediately
                 callback.setValue(null);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
index 2a383b6..5f29981 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobCleanupWork.java
@@ -18,33 +18,27 @@
  */
 package org.apache.hyracks.control.cc.work;
 
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
+import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.api.job.ActivityClusterGraph;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobStatus;
-import org.apache.hyracks.control.cc.ClusterControllerService;
-import org.apache.hyracks.control.cc.NodeControllerState;
-import org.apache.hyracks.control.cc.application.CCApplicationContext;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.common.work.AbstractWork;
 
 public class JobCleanupWork extends AbstractWork {
     private static final Logger LOGGER = Logger.getLogger(JobCleanupWork.class.getName());
 
-    private ClusterControllerService ccs;
+    private IJobManager jobManager;
     private JobId jobId;
     private JobStatus status;
     private List<Exception> exceptions;
 
-    public JobCleanupWork(ClusterControllerService ccs, JobId jobId, JobStatus status, List<Exception> exceptions) {
-        this.ccs = ccs;
+    public JobCleanupWork(IJobManager jobManager, JobId jobId, JobStatus status, List<Exception> exceptions) {
+        this.jobManager = jobManager;
         this.jobId = jobId;
         this.status = status;
         this.exceptions = exceptions;
@@ -52,83 +46,18 @@
 
     @Override
     public void run() {
-        LOGGER.info("Cleanup for JobRun with id: " + jobId);
-        final JobRun run = ccs.getActiveRunMap().get(jobId);
-        if (run == null) {
-            LOGGER.warning("Unable to find JobRun with id: " + jobId);
-            return;
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Cleanup for JobRun with id: " + jobId);
         }
-        if (run.getPendingStatus() != null && run.getCleanupPendingNodeIds().isEmpty()) {
-            finishJob(run);
-            return;
+        try {
+            JobRun jobRun = jobManager.get(jobId);
+            jobManager.prepareComplete(jobRun, status, exceptions);
+        } catch (HyracksException e) {
+            // Fail the job with the caught exception during final completion.
+            JobRun run = jobManager.get(jobId);
+            run.getExceptions().add(e);
+            run.setStatus(JobStatus.FAILURE, run.getExceptions());
         }
-        if (run.getPendingStatus() != null) {
-            LOGGER.warning("Ignoring duplicate cleanup for JobRun with id: " + jobId);
-            return;
-        }
-        Set<String> targetNodes = run.getParticipatingNodeIds();
-        run.getCleanupPendingNodeIds().addAll(targetNodes);
-        if (run.getPendingStatus() != JobStatus.FAILURE && run.getPendingStatus() != JobStatus.TERMINATED) {
-            run.setPendingStatus(status, exceptions);
-        }
-        if (targetNodes != null && !targetNodes.isEmpty()) {
-            Set<String> toDelete = new HashSet<String>();
-            for (String n : targetNodes) {
-                NodeControllerState ncs = ccs.getNodeMap().get(n);
-                try {
-                    if (ncs == null) {
-                        toDelete.add(n);
-                    } else {
-                        ncs.getNodeController().cleanUpJoblet(jobId, status);
-                    }
-                } catch (Exception e) {
-                    e.printStackTrace();
-                }
-            }
-            targetNodes.removeAll(toDelete);
-            run.getCleanupPendingNodeIds().removeAll(toDelete);
-            if (run.getCleanupPendingNodeIds().isEmpty()) {
-                finishJob(run);
-            }
-        } else {
-            finishJob(run);
-        }
-    }
-
-    private void finishJob(final JobRun run) {
-        CCApplicationContext appCtx = ccs.getApplicationContext();
-        if (appCtx != null) {
-            try {
-                appCtx.notifyJobFinish(jobId);
-            } catch (HyracksException e) {
-                e.printStackTrace();
-            }
-        }
-        run.setStatus(run.getPendingStatus(), run.getPendingExceptions());
-        run.setEndTime(System.currentTimeMillis());
-        ccs.getActiveRunMap().remove(jobId);
-        ccs.getRunMapArchive().put(jobId, run);
-        ccs.getRunHistory().put(jobId, run.getExceptions());
-
-        if (run.getActivityClusterGraph().isReportTaskDetails()) {
-            /**
-             * log job details when profiling is enabled
-             */
-            try {
-                ccs.getJobLogFile().log(createJobLogObject(run));
-            } catch (Exception e) {
-                throw new RuntimeException(e);
-            }
-        }
-    }
-
-    private ObjectNode createJobLogObject(final JobRun run) {
-        ObjectMapper om = new ObjectMapper();
-        ObjectNode jobLogObject = om.createObjectNode();
-        ActivityClusterGraph acg = run.getActivityClusterGraph();
-        jobLogObject.set("activity-cluster-graph", acg.toJSON());
-        jobLogObject.set("job-run", run.toJSON());
-        return jobLogObject;
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java
index e7844e9..fefd3b6 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobStartWork.java
@@ -18,7 +18,6 @@
  */
 package org.apache.hyracks.control.cc.work;
 
-import java.util.Collections;
 import java.util.EnumSet;
 
 import org.apache.hyracks.api.deployment.DeploymentId;
@@ -26,9 +25,9 @@
 import org.apache.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.application.CCApplicationContext;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.common.deployment.DeploymentUtils;
 import org.apache.hyracks.control.common.work.IResultCallback;
@@ -54,24 +53,14 @@
 
     @Override
     protected void doRun() throws Exception {
+        IJobManager jobManager = ccs.getJobManager();
         try {
             final CCApplicationContext appCtx = ccs.getApplicationContext();
             IActivityClusterGraphGeneratorFactory acggf = (IActivityClusterGraphGeneratorFactory) DeploymentUtils
                     .deserialize(acggfBytes, deploymentId, appCtx);
             IActivityClusterGraphGenerator acgg = acggf.createActivityClusterGraphGenerator(jobId, appCtx, jobFlags);
-            JobRun run = new JobRun(ccs, deploymentId, jobId, acgg, jobFlags);
-            run.setStatus(JobStatus.INITIALIZED, null);
-            run.setStartTime(System.currentTimeMillis());
-            ccs.getActiveRunMap().put(jobId, run);
-            appCtx.notifyJobCreation(jobId, acggf);
-            run.setStatus(JobStatus.RUNNING, null);
-            try {
-                run.getScheduler().startJob();
-            } catch (Exception e) {
-                ccs.getWorkQueue().schedule(
-                        new JobCleanupWork(ccs, run.getJobId(), JobStatus.FAILURE, Collections.singletonList(e)));
-            }
-            callback.setValue(jobId);
+            JobRun run = new JobRun(ccs, deploymentId, jobId, acggf, acgg, jobFlags, callback);
+            jobManager.add(run);
         } catch (Exception e) {
             callback.setException(e);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
index 603b6f8..6a8e631 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/JobletCleanupNotificationWork.java
@@ -24,9 +24,11 @@
 
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
-import org.apache.hyracks.control.cc.application.CCApplicationContext;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 
 public class JobletCleanupNotificationWork extends AbstractHeartbeatWork {
@@ -45,33 +47,29 @@
 
     @Override
     public void runWork() {
-        final JobRun run = ccs.getActiveRunMap().get(jobId);
+        IJobManager jobManager = ccs.getJobManager();
+        final JobRun run = jobManager.get(jobId);
         Set<String> cleanupPendingNodes = run.getCleanupPendingNodeIds();
         if (!cleanupPendingNodes.remove(nodeId)) {
             if (LOGGER.isLoggable(Level.WARNING)) {
-                LOGGER.warning(nodeId + " not in pending cleanup nodes set: " + cleanupPendingNodes + " for Job: "
-                        + jobId);
+                LOGGER.warning(
+                        nodeId + " not in pending cleanup nodes set: " + cleanupPendingNodes + " for Job: " + jobId);
             }
             return;
         }
-        NodeControllerState ncs = ccs.getNodeMap().get(nodeId);
+        INodeManager nodeManager = ccs.getNodeManager();
+        NodeControllerState ncs = nodeManager.getNodeControllerState(nodeId);
         if (ncs != null) {
             ncs.getActiveJobIds().remove(jobId);
         }
         if (cleanupPendingNodes.isEmpty()) {
-            CCApplicationContext appCtx = ccs.getApplicationContext();
-            if (appCtx != null) {
-                try {
-                    appCtx.notifyJobFinish(jobId);
-                } catch (HyracksException e) {
-                    e.printStackTrace();
-                }
+            try {
+                jobManager.finalComplete(run);
+            } catch (HyracksException e) {
+                // Fail the job with the caught exception during final completion.
+                run.getExceptions().add(e);
+                run.setStatus(JobStatus.FAILURE, run.getExceptions());
             }
-            run.setStatus(run.getPendingStatus(), run.getPendingExceptions());
-            run.setEndTime(System.currentTimeMillis());
-            ccs.getActiveRunMap().remove(jobId);
-            ccs.getRunMapArchive().put(jobId, run);
-            ccs.getRunHistory().put(jobId, run.getExceptions());
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NotifyDeployBinaryWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NotifyDeployBinaryWork.java
index f4b80b1..62c19bb 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NotifyDeployBinaryWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NotifyDeployBinaryWork.java
@@ -48,7 +48,7 @@
 
     @Override
     public void runWork() {
-        /** triggered remotely by a NC to notify that the NC is deployed */
+        // Triggered remotely by a NC to notify that the NC is deployed.
         DeploymentRun dRun = ccs.getDeploymentRun(deploymentId);
         dRun.notifyDeploymentStatus(nodeId, deploymentStatus);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NotifyShutdownWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NotifyShutdownWork.java
index 1efbc6a..5119022 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NotifyShutdownWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/NotifyShutdownWork.java
@@ -39,7 +39,7 @@
 
     @Override
     public void doRun() {
-        /** triggered remotely by a NC to notify that the NC is shutting down */
+        // Triggered remotely by a NC to notify that the NC is shutting down.
         ShutdownRun sRun = ccs.getShutdownRun();
         LOGGER.info("Received shutdown acknowledgement from NC ID:" + nodeId);
         sRun.notifyShutdown(nodeId);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
index dd26ea4..dc93515 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
@@ -18,16 +18,14 @@
  */
 package org.apache.hyracks.control.cc.work;
 
-import java.net.InetAddress;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
-import java.util.Set;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.base.INodeController;
 import org.apache.hyracks.control.common.controllers.NodeParameters;
 import org.apache.hyracks.control.common.controllers.NodeRegistration;
@@ -50,34 +48,15 @@
     @Override
     protected void doRun() throws Exception {
         String id = reg.getNodeId();
-
         IIPCHandle ncIPCHandle = ccs.getClusterIPC().getHandle(reg.getNodeControllerAddress());
-        CCNCFunctions.NodeRegistrationResult result = null;
-        Map<String, String> ncConfiguration = null;
+        CCNCFunctions.NodeRegistrationResult result;
+        Map<String, String> ncConfiguration = new HashMap<>();
         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<InetAddress, Set<String>> ipAddressNodeNameMap = ccs.getIpAddressNodeNameMap();
-            // QQQ Breach of encapsulation here - way too much duplicated data
-            // in NodeRegistration
-            String ipAddress = state.getNCConfig().dataIPAddress;
-            if (state.getNCConfig().dataPublicIPAddress != null) {
-                ipAddress = state.getNCConfig().dataPublicIPAddress;
-            }
-            ncConfiguration = new HashMap<String, String>();
+            INodeManager nodeManager = ccs.getNodeManager();
+            nodeManager.addNode(id, state);
             state.getNCConfig().toMap(ncConfiguration);
-            Set<String> nodes = ipAddressNodeNameMap.get(InetAddress.getByName(ipAddress));
-            if (nodes == null) {
-                nodes = new HashSet<String>();
-                ipAddressNodeNameMap.put(InetAddress.getByName(ipAddress), nodes);
-            }
-            nodes.add(id);
             LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
             NodeParameters params = new NodeParameters();
             params.setClusterControllerInfo(ccs.getClusterControllerInfo());
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java
index 2c5c965..edc57fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterPartitionAvailibilityWork.java
@@ -21,9 +21,9 @@
 import java.util.List;
 
 import org.apache.commons.lang3.tuple.Pair;
-
 import org.apache.hyracks.api.partitions.PartitionId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.cc.partitions.PartitionMatchMaker;
 import org.apache.hyracks.control.cc.partitions.PartitionUtils;
@@ -43,7 +43,8 @@
     @Override
     public void run() {
         final PartitionId pid = partitionDescriptor.getPartitionId();
-        JobRun run = ccs.getActiveRunMap().get(pid.getJobId());
+        IJobManager jobManager = ccs.getJobManager();
+        JobRun run = jobManager.get(pid.getJobId());
         if (run == null) {
             return;
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterPartitionRequestWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterPartitionRequestWork.java
index 44fc40d..100cae2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterPartitionRequestWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterPartitionRequestWork.java
@@ -19,9 +19,9 @@
 package org.apache.hyracks.control.cc.work;
 
 import org.apache.commons.lang3.tuple.Pair;
-
 import org.apache.hyracks.api.partitions.PartitionId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.cc.partitions.PartitionMatchMaker;
 import org.apache.hyracks.control.cc.partitions.PartitionUtils;
@@ -41,7 +41,8 @@
     @Override
     public void run() {
         PartitionId pid = partitionRequest.getPartitionId();
-        JobRun run = ccs.getActiveRunMap().get(pid.getJobId());
+        IJobManager jobManager = ccs.getJobManager();
+        JobRun run = jobManager.get(pid.getJobId());
         if (run == null) {
             return;
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java
index 510c729..410b75f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RemoveDeadNodesWork.java
@@ -18,16 +18,16 @@
  */
 package org.apache.hyracks.control.cc.work;
 
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
+import java.util.Collection;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
-import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.common.work.AbstractWork;
 
@@ -42,40 +42,29 @@
 
     @Override
     public void run() {
-        final Set<String> deadNodes = new HashSet<String>();
-        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
-        for (Map.Entry<String, NodeControllerState> e : nodeMap.entrySet()) {
-            NodeControllerState state = e.getValue();
-            if (state.incrementLastHeartbeatDuration() >= ccs.getCCConfig().maxHeartbeatLapsePeriods) {
-                deadNodes.add(e.getKey());
-                LOGGER.info(e.getKey() + " considered dead");
-            }
-        }
-        Set<JobId> affectedJobIds = new HashSet<JobId>();
-        for (String deadNode : deadNodes) {
-            NodeControllerState state = nodeMap.remove(deadNode);
-
-            // Deal with dead tasks.
-            affectedJobIds.addAll(state.getActiveJobIds());
-        }
-        int size = affectedJobIds.size();
-        if (size > 0) {
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Number of affected jobs: " + size);
-            }
-            for (JobId jobId : affectedJobIds) {
-                JobRun run = ccs.getActiveRunMap().get(jobId);
-                if (run != null) {
-                    run.getScheduler().notifyNodeFailures(deadNodes);
+        try {
+            INodeManager nodeManager = ccs.getNodeManager();
+            Pair<Collection<String>, Collection<JobId>> result = nodeManager.removeDeadNodes();
+            Collection<String> deadNodes = result.getLeft();
+            Collection<JobId> affectedJobIds = result.getRight();
+            int size = affectedJobIds.size();
+            if (size > 0) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("Number of affected jobs: " + size);
+                }
+                IJobManager jobManager = ccs.getJobManager();
+                for (JobId jobId : affectedJobIds) {
+                    JobRun run = jobManager.get(jobId);
+                    if (run != null) {
+                        run.getExecutor().notifyNodeFailures(deadNodes);
+                    }
                 }
             }
-        }
-        if (!deadNodes.isEmpty()) {
-            try {
+            if (!deadNodes.isEmpty()) {
                 ccs.getApplicationContext().notifyNodeFailure(deadNodes);
-            } catch (HyracksException e) {
-                LOGGER.log(Level.WARNING, "Uncaught exception on notifyNodeFailure", e);
             }
+        } catch (HyracksException e) {
+            LOGGER.log(Level.WARNING, "Uncaught exception on notifyNodeFailure", e);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ReportProfilesWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ReportProfilesWork.java
index 2278389..02806a0 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ReportProfilesWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/ReportProfilesWork.java
@@ -16,32 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.control.cc.work;
 
 import java.util.List;
-import java.util.Map;
 import java.util.logging.Level;
 
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.common.job.profiling.om.JobProfile;
 import org.apache.hyracks.control.common.work.AbstractWork;
 
 public class ReportProfilesWork extends AbstractWork {
-    private final ClusterControllerService ccs;
+    private final IJobManager jobManager;
     private final List<JobProfile> profiles;
 
-    public ReportProfilesWork(ClusterControllerService ccs, List<JobProfile> profiles) {
-        this.ccs = ccs;
+    public ReportProfilesWork(IJobManager jobManager, List<JobProfile> profiles) {
+        this.jobManager = jobManager;
         this.profiles = profiles;
     }
 
     @Override
     public void run() {
-        Map<JobId, JobRun> runMap = ccs.getActiveRunMap();
         for (JobProfile profile : profiles) {
-            JobRun run = runMap.get(profile.getJobId());
+            JobRun run = jobManager.get(profile.getJobId());
             if (run != null) {
                 JobProfile jp = run.getJobProfile();
                 jp.merge(profile);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
index 2379871..f4f2f52 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskCompleteWork.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.cc.job.TaskAttempt;
 import org.apache.hyracks.control.common.job.profiling.om.JobProfile;
@@ -42,7 +43,8 @@
     @Override
     protected void performEvent(TaskAttempt ta) {
         try {
-            JobRun run = ccs.getActiveRunMap().get(jobId);
+            IJobManager jobManager = ccs.getJobManager();
+            JobRun run = jobManager.get(jobId);
             if (statistics != null) {
                 JobProfile jobProfile = run.getJobProfile();
                 Map<String, JobletProfile> jobletProfiles = jobProfile.getJobletProfiles();
@@ -53,7 +55,7 @@
                 }
                 jobletProfile.getTaskProfiles().put(taId, statistics);
             }
-            run.getScheduler().notifyTaskComplete(ta);
+            run.getExecutor().notifyTaskComplete(ta);
         } catch (HyracksException e) {
             e.printStackTrace();
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java
index 8bca4e7..486e9c6 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/TaskFailureWork.java
@@ -23,6 +23,7 @@
 import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.JobRun;
 import org.apache.hyracks.control.cc.job.TaskAttempt;
 
@@ -37,9 +38,10 @@
 
     @Override
     protected void performEvent(TaskAttempt ta) {
-        JobRun run = ccs.getActiveRunMap().get(jobId);
+        IJobManager jobManager = ccs.getJobManager();
+        JobRun run = jobManager.get(jobId);
         ccs.getDatasetDirectoryService().reportJobFailure(jobId, exceptions);
-        run.getScheduler().notifyTaskFailure(ta, exceptions);
+        run.getExecutor().notifyTaskFailure(ta, exceptions);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UnregisterNodeWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UnregisterNodeWork.java
index 839dc45..7eb5345 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UnregisterNodeWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UnregisterNodeWork.java
@@ -18,24 +18,20 @@
  */
 package org.apache.hyracks.control.cc.work;
 
-import java.util.Map;
-
-import org.apache.hyracks.control.cc.ClusterControllerService;
-import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 
 public class UnregisterNodeWork extends SynchronizableWork {
-    private final ClusterControllerService ccs;
+    private final INodeManager nodeManager;
     private final String nodeId;
 
-    public UnregisterNodeWork(ClusterControllerService ccs, String nodeId) {
-        this.ccs = ccs;
+    public UnregisterNodeWork(INodeManager nodeManager, String nodeId) {
+        this.nodeManager = nodeManager;
         this.nodeId = nodeId;
     }
 
     @Override
     protected void doRun() throws Exception {
-        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
-        nodeMap.remove(nodeId);
+        nodeManager.removeNode(nodeId);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
index c1fa945..f7ef175 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/WaitForJobCompletionWork.java
@@ -22,6 +22,7 @@
 
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.job.IJobManager;
 import org.apache.hyracks.control.cc.job.IJobStatusConditionVariable;
 import org.apache.hyracks.control.common.work.IResultCallback;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
@@ -39,7 +40,8 @@
 
     @Override
     protected void doRun() throws Exception {
-        final IJobStatusConditionVariable cRunningVar = ccs.getActiveRunMap().get(jobId);
+        IJobManager jobManager = ccs.getJobManager();
+        final IJobStatusConditionVariable cRunningVar = jobManager.get(jobId);
         if (cRunningVar != null) {
             ccs.getExecutor().execute(new Runnable() {
                 @Override
@@ -53,32 +55,20 @@
                 }
             });
         } else {
-            final IJobStatusConditionVariable cArchivedVar = ccs.getRunMapArchive().get(jobId);
-            if (cArchivedVar != null) {
-                ccs.getExecutor().execute(new Runnable() {
-                    @Override
-                    public void run() {
-                        try {
-                            cArchivedVar.waitForCompletion();
-                            callback.setValue(null);
-                        } catch (Exception e) {
-                            callback.setException(e);
-                        }
+            final List<Exception> exceptions = jobManager.getRunHistory(jobId);
+            ccs.getExecutor().execute(new Runnable() {
+                @Override
+                public void run() {
+                    callback.setValue(null);
+                    if (exceptions != null && !exceptions.isEmpty()) {
+                        /**
+                         * only report the first exception because IResultCallback will only throw one exception
+                         * anyway
+                         */
+                        callback.setException(exceptions.get(0));
                     }
-                });
-            } else {
-                final List<Exception> exceptions = ccs.getRunHistory().get(jobId);
-                ccs.getExecutor().execute(new Runnable() {
-                    @Override
-                    public void run() {
-                        callback.setValue(null);
-                        if (exceptions != null && exceptions.size() > 0) {
-                            /** only report the first exception because IResultCallback will only throw one exception anyway */
-                            callback.setException(exceptions.get(0));
-                        }
-                    }
-                });
-            }
+                }
+            });
         }
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
new file mode 100644
index 0000000..c742a4a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.control.cc.cluster;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.resource.IReadOnlyClusterCapacity;
+import org.apache.hyracks.api.job.resource.NodeCapacity;
+import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.scheduler.IResourceManager;
+import org.apache.hyracks.control.cc.scheduler.ResourceManager;
+import org.apache.hyracks.control.common.controllers.CCConfig;
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class NodeManagerTest {
+
+    private static final long NODE_MEMORY_SIZE = 1024L;
+    private static final int NODE_CORES = 8;
+    private static final String NODE1 = "node1";
+    private static final String NODE2 = "node2";
+
+    @Test
+    public void testNormal() throws HyracksException {
+        IResourceManager resourceManager = new ResourceManager();
+        INodeManager nodeManager = new NodeManager(makeCCConfig(), resourceManager);
+        NodeControllerState ncState1 = mockNodeControllerState(false);
+        NodeControllerState ncState2 = mockNodeControllerState(false);
+
+        // Verifies states after adding nodes.
+        nodeManager.addNode(NODE1, ncState1);
+        nodeManager.addNode(NODE2, ncState2);
+        Assert.assertTrue(nodeManager.getIpAddressNodeNameMap().size() == 1);
+        Assert.assertTrue(nodeManager.getAllNodeIds().size() == 2);
+        Assert.assertTrue(nodeManager.getAllNodeControllerStates().size() == 2);
+        Assert.assertTrue(nodeManager.getNodeControllerState(NODE1) == ncState1);
+        Assert.assertTrue(nodeManager.getNodeControllerState(NODE2) == ncState2);
+        Assert.assertTrue(resourceManager.getCurrentCapacity().getAggregatedMemoryByteSize() == NODE_MEMORY_SIZE * 2);
+        Assert.assertTrue(resourceManager.getCurrentCapacity().getAggregatedCores() == NODE_CORES * 2);
+        Assert.assertTrue(resourceManager.getMaximumCapacity().getAggregatedMemoryByteSize() == NODE_MEMORY_SIZE * 2);
+        Assert.assertTrue(resourceManager.getMaximumCapacity().getAggregatedCores() == NODE_CORES * 2);
+
+        // Verifies states after removing dead nodes.
+        nodeManager.removeDeadNodes();
+        verifyEmptyCluster(resourceManager, nodeManager);
+    }
+
+    @Test
+    public void testException() throws HyracksException {
+        IResourceManager resourceManager = new ResourceManager();
+        INodeManager nodeManager = new NodeManager(makeCCConfig(), resourceManager);
+        NodeControllerState ncState1 = mockNodeControllerState(true);
+
+        boolean invalidNetworkAddress = false;
+        // Verifies states after a failure during adding nodes.
+        try {
+            nodeManager.addNode(NODE1, ncState1);
+        } catch (HyracksException e) {
+            invalidNetworkAddress = e.getErrorCode() == ErrorCode.INVALID_NETWORK_ADDRESS;
+        }
+        Assert.assertTrue(invalidNetworkAddress);
+
+        // Verifies that the cluster is empty.
+        verifyEmptyCluster(resourceManager, nodeManager);
+    }
+
+    @Test
+    public void testNullNode() throws HyracksException {
+        IResourceManager resourceManager = new ResourceManager();
+        INodeManager nodeManager = new NodeManager(makeCCConfig(), resourceManager);
+
+        boolean invalidParameter = false;
+        // Verifies states after a failure during adding nodes.
+        try {
+            nodeManager.addNode(null, null);
+        } catch (HyracksException e) {
+            invalidParameter = e.getErrorCode() == ErrorCode.INVALID_INPUT_PARAMETER;
+        }
+        Assert.assertTrue(invalidParameter);
+
+        // Verifies that the cluster is empty.
+        verifyEmptyCluster(resourceManager, nodeManager);
+    }
+
+    private CCConfig makeCCConfig() {
+        CCConfig ccConfig = new CCConfig();
+        ccConfig.maxHeartbeatLapsePeriods = 0;
+        return ccConfig;
+    }
+
+    private NodeControllerState mockNodeControllerState(boolean invalidIpAddr) {
+        NodeControllerState ncState = mock(NodeControllerState.class);
+        String ipAddr = invalidIpAddr ? "255.255.255:255" : "127.0.0.2";
+        NetworkAddress dataAddr = new NetworkAddress(ipAddr, 1001);
+        NetworkAddress resultAddr = new NetworkAddress(ipAddr, 1002);
+        NetworkAddress msgAddr = new NetworkAddress(ipAddr, 1003);
+        when(ncState.getCapacity()).thenReturn(new NodeCapacity(NODE_MEMORY_SIZE, NODE_CORES));
+        when(ncState.getDataPort()).thenReturn(dataAddr);
+        when(ncState.getDatasetPort()).thenReturn(resultAddr);
+        when(ncState.getMessagingPort()).thenReturn(msgAddr);
+        NCConfig ncConfig = new NCConfig();
+        ncConfig.dataIPAddress = ipAddr;
+        when(ncState.getNCConfig()).thenReturn(ncConfig);
+        return ncState;
+    }
+
+    private void verifyEmptyCluster(IResourceManager resourceManager, INodeManager nodeManager) {
+        Assert.assertTrue(nodeManager.getIpAddressNodeNameMap().isEmpty());
+        Assert.assertTrue(nodeManager.getAllNodeIds().isEmpty());
+        Assert.assertTrue(nodeManager.getAllNodeControllerStates().isEmpty());
+        Assert.assertTrue(nodeManager.getNodeControllerState(NODE1) == null);
+        Assert.assertTrue(nodeManager.getNodeControllerState(NODE2) == null);
+
+        IReadOnlyClusterCapacity currentCapacity = resourceManager.getCurrentCapacity();
+        IReadOnlyClusterCapacity maximumCapacity = resourceManager.getMaximumCapacity();
+        Assert.assertTrue(currentCapacity.getAggregatedMemoryByteSize() == 0L);
+        Assert.assertTrue(currentCapacity.getAggregatedCores() == 0);
+        Assert.assertTrue(maximumCapacity.getAggregatedMemoryByteSize() == 0L);
+        Assert.assertTrue(maximumCapacity.getAggregatedCores() == 0);
+        verifyNodeNotExistInCapacity(currentCapacity, NODE1);
+        verifyNodeNotExistInCapacity(currentCapacity, NODE2);
+        verifyNodeNotExistInCapacity(maximumCapacity, NODE1);
+        verifyNodeNotExistInCapacity(maximumCapacity, NODE1);
+    }
+
+    private void verifyNodeNotExistInCapacity(IReadOnlyClusterCapacity capacity, String nodeId) {
+        boolean nodeNotExist = false;
+        try {
+            capacity.getMemoryByteSize(nodeId);
+        } catch (HyracksException e) {
+            nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+        }
+        Assert.assertTrue(nodeNotExist);
+        nodeNotExist = false;
+        try {
+            capacity.getCores(nodeId);
+        } catch (HyracksException e) {
+            nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+        }
+        Assert.assertTrue(nodeNotExist);
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/job/JobManagerTest.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/job/JobManagerTest.java
new file mode 100644
index 0000000..5e1b856
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/job/JobManagerTest.java
@@ -0,0 +1,238 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.control.cc.job;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.JobStatus;
+import org.apache.hyracks.api.job.resource.IJobCapacityController;
+import org.apache.hyracks.control.cc.ClusterControllerService;
+import org.apache.hyracks.control.cc.NodeControllerState;
+import org.apache.hyracks.control.cc.application.CCApplicationContext;
+import org.apache.hyracks.control.cc.cluster.INodeManager;
+import org.apache.hyracks.control.cc.cluster.NodeManager;
+import org.apache.hyracks.control.common.base.INodeController;
+import org.apache.hyracks.control.common.controllers.CCConfig;
+import org.apache.hyracks.control.common.logs.LogFile;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class JobManagerTest {
+
+    @Test
+    public void test() throws HyracksException {
+        CCConfig ccConfig = new CCConfig();
+        IJobCapacityController jobCapacityController = mock(IJobCapacityController.class);
+        IJobManager jobManager = spy(new JobManager(ccConfig, mockClusterControllerService(), jobCapacityController));
+
+        // Submits runnable jobs.
+        List<JobRun> acceptedRuns = new ArrayList<>();
+        for (int id = 0; id < 4096; ++id) {
+            // Mocks an immediately executable job.
+            JobRun run = mockJobRun(id);
+            JobSpecification job = mock(JobSpecification.class);
+            when(run.getActivityClusterGraphFactory().getJobSpecification()).thenReturn(job);
+            when(jobCapacityController.allocate(job)).thenReturn(IJobCapacityController.JobSubmissionStatus.EXECUTE);
+
+            // Submits the job.
+            acceptedRuns.add(run);
+            jobManager.add(run);
+            Assert.assertTrue(jobManager.getRunningJobs().size() == id + 1);
+            Assert.assertTrue(jobManager.getPendingJobs().isEmpty());
+        }
+
+        // Submits jobs that will be deferred due to the capacity limitation.
+        List<JobRun> deferredRuns = new ArrayList<>();
+        for (int id = 4096; id < 8192; ++id) {
+            // Mocks a deferred job.
+            JobRun run = mockJobRun(id);
+            JobSpecification job = mock(JobSpecification.class);
+            when(run.getActivityClusterGraphFactory().getJobSpecification()).thenReturn(job);
+            when(jobCapacityController.allocate(job)).thenReturn(IJobCapacityController.JobSubmissionStatus.QUEUE)
+                    .thenReturn(IJobCapacityController.JobSubmissionStatus.EXECUTE);
+
+            // Submits the job.
+            deferredRuns.add(run);
+            jobManager.add(run);
+            Assert.assertTrue(jobManager.getRunningJobs().size() == 4096);
+            Assert.assertTrue(jobManager.getPendingJobs().size() == id + 1 - 4096);
+        }
+
+        // Further jobs will be denied because the job queue is full.
+        boolean jobQueueFull = false;
+        try {
+            JobRun run = mockJobRun(8193);
+            JobSpecification job = mock(JobSpecification.class);
+            when(run.getActivityClusterGraphFactory().getJobSpecification()).thenReturn(job);
+            when(jobCapacityController.allocate(job)).thenReturn(IJobCapacityController.JobSubmissionStatus.QUEUE)
+                    .thenReturn(IJobCapacityController.JobSubmissionStatus.EXECUTE);
+            jobManager.add(run);
+        } catch (HyracksException e) {
+            // Verifies the error code.
+            jobQueueFull = e.getErrorCode() == ErrorCode.JOB_QUEUE_FULL;
+        }
+        Assert.assertTrue(jobQueueFull);
+
+        // Completes runnable jobs.
+        for (JobRun run : acceptedRuns) {
+            jobManager.prepareComplete(run, JobStatus.TERMINATED, Collections.emptyList());
+            jobManager.finalComplete(run);
+        }
+        Assert.assertTrue(jobManager.getRunningJobs().size() == 4096);
+        Assert.assertTrue(jobManager.getPendingJobs().isEmpty());
+        Assert.assertTrue(jobManager.getArchivedJobs().size() == ccConfig.jobHistorySize);
+
+        // Completes deferred jobs.
+        for (JobRun run : deferredRuns) {
+            jobManager.prepareComplete(run, JobStatus.TERMINATED, Collections.emptyList());
+            jobManager.finalComplete(run);
+        }
+        Assert.assertTrue(jobManager.getRunningJobs().isEmpty());
+        Assert.assertTrue(jobManager.getPendingJobs().isEmpty());
+        Assert.assertTrue(jobManager.getArchivedJobs().size() == ccConfig.jobHistorySize);
+        verify(jobManager, times(8192)).prepareComplete(any(), any(), any());
+        verify(jobManager, times(8192)).finalComplete(any());
+    }
+
+    @Test
+    public void testExceedMax() throws HyracksException {
+        CCConfig ccConfig = new CCConfig();
+        IJobCapacityController jobCapacityController = mock(IJobCapacityController.class);
+        IJobManager jobManager = spy(new JobManager(ccConfig, mockClusterControllerService(), jobCapacityController));
+        boolean rejected = false;
+        // A job should be rejected immediately if its requirement exceeds the maximum capacity of the cluster.
+        try {
+            JobRun run = mockJobRun(1);
+            JobSpecification job = mock(JobSpecification.class);
+            when(run.getActivityClusterGraphFactory().getJobSpecification()).thenReturn(job);
+            when(jobCapacityController.allocate(job))
+                    .thenThrow(HyracksException.create(ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY, "1", "0"));
+            jobManager.add(run);
+        } catch (HyracksException e) {
+            // Verifies the error code.
+            rejected = e.getErrorCode() == ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY;
+        }
+        Assert.assertTrue(rejected);
+        Assert.assertTrue(jobManager.getRunningJobs().isEmpty());
+        Assert.assertTrue(jobManager.getPendingJobs().isEmpty());
+        Assert.assertTrue(jobManager.getArchivedJobs().size() == 0);
+    }
+
+    @Test
+    public void testAdmitThenReject() throws HyracksException {
+        CCConfig ccConfig = new CCConfig();
+        IJobCapacityController jobCapacityController = mock(IJobCapacityController.class);
+        IJobManager jobManager = spy(new JobManager(ccConfig, mockClusterControllerService(), jobCapacityController));
+
+        // A pending job should also be rejected if its requirement exceeds the updated maximum capacity of the cluster.
+        // A normal run.
+        JobRun run1 = mockJobRun(1);
+        JobSpecification job1 = mock(JobSpecification.class);
+        when(run1.getActivityClusterGraphFactory().getJobSpecification()).thenReturn(job1);
+        when(jobCapacityController.allocate(job1)).thenReturn(IJobCapacityController.JobSubmissionStatus.EXECUTE);
+        jobManager.add(run1);
+
+        // A failure run.
+        JobRun run2 = mockJobRun(2);
+        JobSpecification job2 = mock(JobSpecification.class);
+        when(run2.getActivityClusterGraphFactory().getJobSpecification()).thenReturn(job2);
+        when(jobCapacityController.allocate(job2)).thenReturn(IJobCapacityController.JobSubmissionStatus.QUEUE)
+                .thenThrow(HyracksException.create(ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY, "1", "0"));
+        jobManager.add(run2);
+
+        // Completes the first run.
+        jobManager.prepareComplete(run1, JobStatus.TERMINATED, Collections.emptyList());
+        jobManager.finalComplete(run1);
+
+        // Verifies job status of the failed job.
+        verify(run2, times(1)).setStatus(eq(JobStatus.PENDING), any());
+        verify(run2, times(1)).setPendingStatus(eq(JobStatus.FAILURE), any());
+    }
+
+    @Test
+    public void testNullJob() throws HyracksException {
+        CCConfig ccConfig = new CCConfig();
+        IJobCapacityController jobCapacityController = mock(IJobCapacityController.class);
+        IJobManager jobManager = new JobManager(ccConfig, mockClusterControllerService(), jobCapacityController);
+        boolean invalidParameter = false;
+        try {
+            jobManager.add(null);
+        } catch (HyracksException e) {
+            invalidParameter = e.getErrorCode() == ErrorCode.INVALID_INPUT_PARAMETER;
+        }
+        Assert.assertTrue(invalidParameter);
+        Assert.assertTrue(jobManager.getRunningJobs().isEmpty());
+        Assert.assertTrue(jobManager.getPendingJobs().isEmpty());
+    }
+
+    private JobRun mockJobRun(long id) {
+        JobRun run = mock(JobRun.class, Mockito.RETURNS_DEEP_STUBS);
+        when(run.getExceptions()).thenReturn(Collections.emptyList());
+        when(run.getActivityClusterGraph().isReportTaskDetails()).thenReturn(true);
+        when(run.getPendingExceptions()).thenReturn(Collections.emptyList());
+        JobId jobId = new JobId(id);
+        when(run.getJobId()).thenReturn(jobId);
+
+        Set<String> nodes = new HashSet<>();
+        nodes.add("node1");
+        nodes.add("node2");
+        when(run.getParticipatingNodeIds()).thenReturn(nodes);
+        when(run.getCleanupPendingNodeIds()).thenReturn(nodes);
+        return run;
+    }
+
+    private ClusterControllerService mockClusterControllerService() {
+        ClusterControllerService ccs = mock(ClusterControllerService.class);
+        CCApplicationContext appCtx = mock(CCApplicationContext.class);
+        LogFile logFile = mock(LogFile.class);
+        INodeManager nodeManager = mockNodeManager();
+        when(ccs.getApplicationContext()).thenReturn(appCtx);
+        when(ccs.getJobLogFile()).thenReturn(logFile);
+        when(ccs.getNodeManager()).thenReturn(nodeManager);
+        return ccs;
+    }
+
+    private INodeManager mockNodeManager() {
+        INodeManager nodeManager = mock(NodeManager.class);
+        NodeControllerState ncState = mock(NodeControllerState.class);
+        INodeController nodeController = mock(INodeController.class);
+        when(nodeManager.getNodeControllerState(any())).thenReturn(ncState);
+        when(ncState.getNodeController()).thenReturn(nodeController);
+        return nodeManager;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
index dff5827..5c27a6f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
@@ -22,6 +22,7 @@
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.dataflow.ConnectorDescriptorId;
@@ -37,7 +38,7 @@
 public interface INodeController {
     public void startTasks(DeploymentId deploymentId, JobId jobId, byte[] planBytes,
             List<TaskAttemptDescriptor> taskDescriptors, Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies,
-            EnumSet<JobFlag> flags) throws Exception;
+            Set<JobFlag> flags) throws Exception;
 
     public void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception;
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
index d7c3d36..b636a096 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
@@ -102,6 +102,15 @@
             usage = "Specify path to master configuration file (default: none)", required = false)
     public String configFile = null;
 
+    @Option(name = "-job-queue-class-name", usage = "Specify the implementation class name for the job queue. (default:"
+            + " org.apache.hyracks.control.cc.scheduler.FIFOJobQueue)",
+            required = false)
+    public String jobQueueClassName = "org.apache.hyracks.control.cc.scheduler.FIFOJobQueue";
+
+    @Option(name = "-job-manager-class-name", usage = "Specify the implementation class name for the job manager. "
+            + "(default: org.apache.hyracks.control.cc.job.JobManager)", required = false)
+    public String jobManagerClassName = "org.apache.hyracks.control.cc.job.JobManager";
+
     @Argument
     @Option(name = "--", handler = StopOptionHandler.class)
     public List<String> appArgs;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
index e95a004..490b6ff 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NodeRegistration.java
@@ -24,6 +24,7 @@
 import java.util.Map;
 
 import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.job.resource.NodeCapacity;
 import org.apache.hyracks.control.common.heartbeat.HeartbeatSchema;
 
 public final class NodeRegistration implements Serializable {
@@ -69,11 +70,13 @@
 
     private final int pid;
 
+    private final NodeCapacity capacity;
+
     public NodeRegistration(InetSocketAddress ncAddress, String nodeId, NCConfig ncConfig, NetworkAddress dataPort,
                             NetworkAddress datasetPort, String osName, String arch, String osVersion, int nProcessors,
                             String vmName, String vmVersion, String vmVendor, String classpath, String libraryPath,
                             String bootClasspath, List<String> inputArguments, Map<String, String> systemProperties,
-                            HeartbeatSchema hbSchema, NetworkAddress messagingPort, int pid) {
+            HeartbeatSchema hbSchema, NetworkAddress messagingPort, NodeCapacity capacity, int pid) {
         this.ncAddress = ncAddress;
         this.nodeId = nodeId;
         this.ncConfig = ncConfig;
@@ -93,6 +96,7 @@
         this.systemProperties = systemProperties;
         this.hbSchema = hbSchema;
         this.messagingPort = messagingPort;
+        this.capacity = capacity;
         this.pid = pid;
     }
 
@@ -104,6 +108,10 @@
         return nodeId;
     }
 
+    public NodeCapacity getCapacity() {
+        return capacity;
+    }
+
     public NCConfig getNCConfig() {
         return ncConfig;
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentRun.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentRun.java
index 416b0e6..3c5c97a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentRun.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/deployment/DeploymentRun.java
@@ -19,6 +19,7 @@
 
 package org.apache.hyracks.control.common.deployment;
 
+import java.util.Collection;
 import java.util.Set;
 import java.util.TreeSet;
 
@@ -33,7 +34,7 @@
     private DeploymentStatus deploymentStatus = DeploymentStatus.FAIL;
     private final Set<String> deploymentNodeIds = new TreeSet<String>();
 
-    public DeploymentRun(Set<String> nodeIds) {
+    public DeploymentRun(Collection<String> nodeIds) {
         deploymentNodeIds.addAll(nodeIds);
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
index aa9a4fe..4ee34ca 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.control.common.ipc;
 
 import java.io.ByteArrayInputStream;
@@ -33,6 +34,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -676,11 +678,11 @@
         private final byte[] planBytes;
         private final List<TaskAttemptDescriptor> taskDescriptors;
         private final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies;
-        private final EnumSet<JobFlag> flags;
+        private final Set<JobFlag> flags;
 
         public StartTasksFunction(DeploymentId deploymentId, JobId jobId, byte[] planBytes,
                 List<TaskAttemptDescriptor> taskDescriptors,
-                Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, EnumSet<JobFlag> flags) {
+                Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, Set<JobFlag> flags) {
             this.deploymentId = deploymentId;
             this.jobId = jobId;
             this.planBytes = planBytes;
@@ -714,7 +716,7 @@
             return connectorPolicies;
         }
 
-        public EnumSet<JobFlag> getFlags() {
+        public Set<JobFlag> getFlags() {
             return flags;
         }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
index c3376e6..0d59b8d 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
@@ -19,9 +19,9 @@
 package org.apache.hyracks.control.common.ipc;
 
 import java.net.URL;
-import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.dataflow.ConnectorDescriptorId;
@@ -46,7 +46,7 @@
     @Override
     public void startTasks(DeploymentId deploymentId, JobId jobId, byte[] planBytes,
             List<TaskAttemptDescriptor> taskDescriptors, Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies,
-            EnumSet<JobFlag> flags) throws Exception {
+            Set<JobFlag> flags) throws Exception {
         CCNCFunctions.StartTasksFunction stf = new CCNCFunctions.StartTasksFunction(deploymentId, jobId, planBytes,
                 taskDescriptors, connectorPolicies, flags);
         ipcHandle.send(-1, stf, null);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/shutdown/ShutdownRun.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/shutdown/ShutdownRun.java
index 0a50f6f..eae2eb6 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/shutdown/ShutdownRun.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/shutdown/ShutdownRun.java
@@ -19,6 +19,7 @@
 
 package org.apache.hyracks.control.common.shutdown;
 
+import java.util.Collection;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
@@ -29,7 +30,7 @@
     private boolean shutdownSuccess = false;
     private static final long SHUTDOWN_TIMER_MS = TimeUnit.SECONDS.toMillis(30);
 
-    public ShutdownRun(Set<String> nodeIds) {
+    public ShutdownRun(Collection<String> nodeIds) {
         shutdownNodeIds.addAll(nodeIds);
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index d44a4e5..19f01c1 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -49,6 +49,7 @@
 import org.apache.hyracks.api.io.IODeviceHandle;
 import org.apache.hyracks.api.job.ActivityClusterGraph;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.resource.NodeCapacity;
 import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
 import org.apache.hyracks.api.lifecycle.LifeCycleComponentManager;
 import org.apache.hyracks.api.service.IControllerService;
@@ -266,11 +267,15 @@
         NetworkAddress netAddress = netManager.getPublicNetworkAddress();
         NetworkAddress meesagingPort = messagingNetManager != null ? messagingNetManager.getPublicNetworkAddress()
                 : null;
+        int allCores = osMXBean.getAvailableProcessors();
         ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig, netAddress, datasetAddress,
-                osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), osMXBean.getAvailableProcessors(),
+                osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), allCores,
                 runtimeMXBean.getVmName(), runtimeMXBean.getVmVersion(), runtimeMXBean.getVmVendor(),
                 runtimeMXBean.getClassPath(), runtimeMXBean.getLibraryPath(), runtimeMXBean.getBootClassPath(),
                 runtimeMXBean.getInputArguments(), runtimeMXBean.getSystemProperties(), hbSchema, meesagingPort,
+                ncAppEntryPoint == null
+                        ? new NodeCapacity(Runtime.getRuntime().maxMemory(), allCores > 1 ? allCores - 1 : allCores)
+                        : ncAppEntryPoint.getCapacity(),
                 PidHelper.getPid()));
 
         synchronized (this) {
@@ -455,7 +460,7 @@
 
             hbData.diskReads = ioCounter.getReads();
             hbData.diskWrites = ioCounter.getWrites();
-            hbData.numCores = Runtime.getRuntime().availableProcessors();
+            hbData.numCores = Runtime.getRuntime().availableProcessors() - 1; // Reserves one core for heartbeats.
 
             try {
                 cc.nodeHeartbeat(id, hbData);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
index cb38076..05c9f07 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
@@ -343,7 +343,7 @@
     public FileReference resolveAbsolutePath(String path) throws HyracksDataException {
         IODeviceHandle devHandle = getDevice(path);
         if (devHandle == null) {
-            throw HyracksDataException.create(ErrorCode.RUNTIME_FILE_WITH_ABSOULTE_PATH_NOT_WITHIN_ANY_IO_DEVICE, path);
+            throw HyracksDataException.create(ErrorCode.FILE_WITH_ABSOULTE_PATH_NOT_WITHIN_ANY_IO_DEVICE, path);
         }
         String relativePath = devHandle.getRelativePath(path);
         return new FileReference(devHandle, relativePath);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
index d27caf2..803f15a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
@@ -22,9 +22,9 @@
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
-import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -83,11 +83,11 @@
 
     private final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap;
 
-    private final EnumSet<JobFlag> flags;
+    private final Set<JobFlag> flags;
 
     public StartTasksWork(NodeControllerService ncs, DeploymentId deploymentId, JobId jobId, byte[] acgBytes,
             List<TaskAttemptDescriptor> taskDescriptors,
-            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap, EnumSet<JobFlag> flags) {
+            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap, Set<JobFlag> flags) {
         this.ncs = ncs;
         this.deploymentId = deploymentId;
         this.jobId = jobId;
@@ -219,7 +219,7 @@
 
     private IPartitionWriterFactory createPartitionWriterFactory(final IHyracksTaskContext ctx,
             IConnectorPolicy cPolicy, final JobId jobId, final IConnectorDescriptor conn, final int senderIndex,
-            final TaskAttemptId taId, EnumSet<JobFlag> flags) {
+            final TaskAttemptId taId, Set<JobFlag> flags) {
         IPartitionWriterFactory factory;
         if (cPolicy.materializeOnSendSide()) {
             if (cPolicy.consumerWaitsForProducerToFinish()) {
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/NCApplicationEntryPoint.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/NCApplicationEntryPoint.java
index 66ec854..5180f23 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/NCApplicationEntryPoint.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/org/apache/hyracks/examples/btree/helper/NCApplicationEntryPoint.java
@@ -20,8 +20,10 @@
 
 import org.apache.hyracks.api.application.INCApplicationContext;
 import org.apache.hyracks.api.application.INCApplicationEntryPoint;
+import org.apache.hyracks.api.job.resource.NodeCapacity;
 
 public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
+
     @Override
     public void start(INCApplicationContext ncAppCtx, String[] args) throws Exception {
         RuntimeContext rCtx = new RuntimeContext(ncAppCtx);
@@ -37,4 +39,9 @@
     public void stop() throws Exception {
 
     }
+
+    @Override
+    public NodeCapacity getCapacity() {
+        return new NodeCapacity(Runtime.getRuntime().maxMemory(), Runtime.getRuntime().availableProcessors() - 1);
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
index 9e6afa7..e4b220b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
@@ -122,12 +122,12 @@
                 // If it's a list, it can have multiple keywords in it. But, each keyword should not be a phrase.
                 if (isFullTextSearchQuery) {
                     if (queryTokenizerType == TokenizerType.STRING && tokenCountInOneField > 1) {
-                        throw HyracksDataException.create(ErrorCode.RUNTIME_FULLTEXT_PHRASE_FOUND);
+                        throw HyracksDataException.create(ErrorCode.FULLTEXT_PHRASE_FOUND);
                     } else if (queryTokenizerType == TokenizerType.LIST) {
                         for (int j = 1; j < token.getTokenLength(); j++) {
                             if (DelimitedUTF8StringBinaryTokenizer
                                     .isSeparator((char) token.getData()[token.getStartOffset() + j])) {
-                                throw HyracksDataException.create(ErrorCode.RUNTIME_FULLTEXT_PHRASE_FOUND);
+                                throw HyracksDataException.create(ErrorCode.FULLTEXT_PHRASE_FOUND);
                             }
                         }
                     }