|
@@ -34,7 +34,6 @@ import java.util.Random;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
-import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
|
|
import org.apache.commons.lang.StringUtils;
|
|
import org.apache.commons.lang.StringUtils;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
@@ -219,8 +218,6 @@ public class CapacityScheduler extends
|
|
|
|
|
|
private Map<String, CSQueue> queues = new ConcurrentHashMap<String, CSQueue>();
|
|
private Map<String, CSQueue> queues = new ConcurrentHashMap<String, CSQueue>();
|
|
|
|
|
|
- private AtomicInteger numNodeManagers = new AtomicInteger(0);
|
|
|
|
-
|
|
|
|
private ResourceCalculator calculator;
|
|
private ResourceCalculator calculator;
|
|
private boolean usePortForNodeName;
|
|
private boolean usePortForNodeName;
|
|
|
|
|
|
@@ -280,7 +277,7 @@ public class CapacityScheduler extends
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public int getNumClusterNodes() {
|
|
public int getNumClusterNodes() {
|
|
- return numNodeManagers.get();
|
|
|
|
|
|
+ return nodeTracker.nodeCount();
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -387,7 +384,7 @@ public class CapacityScheduler extends
|
|
static void schedule(CapacityScheduler cs) {
|
|
static void schedule(CapacityScheduler cs) {
|
|
// First randomize the start point
|
|
// First randomize the start point
|
|
int current = 0;
|
|
int current = 0;
|
|
- Collection<FiCaSchedulerNode> nodes = cs.getAllNodes().values();
|
|
|
|
|
|
+ Collection<FiCaSchedulerNode> nodes = cs.nodeTracker.getAllNodes();
|
|
int start = random.nextInt(nodes.size());
|
|
int start = random.nextInt(nodes.size());
|
|
for (FiCaSchedulerNode node : nodes) {
|
|
for (FiCaSchedulerNode node : nodes) {
|
|
if (current++ >= start) {
|
|
if (current++ >= start) {
|
|
@@ -524,10 +521,11 @@ public class CapacityScheduler extends
|
|
addNewQueues(queues, newQueues);
|
|
addNewQueues(queues, newQueues);
|
|
|
|
|
|
// Re-configure queues
|
|
// Re-configure queues
|
|
- root.reinitialize(newRoot, clusterResource);
|
|
|
|
|
|
+ root.reinitialize(newRoot, getClusterResource());
|
|
updatePlacementRules();
|
|
updatePlacementRules();
|
|
|
|
|
|
// Re-calculate headroom for active applications
|
|
// Re-calculate headroom for active applications
|
|
|
|
+ Resource clusterResource = getClusterResource();
|
|
root.updateClusterResource(clusterResource, new ResourceLimits(
|
|
root.updateClusterResource(clusterResource, new ResourceLimits(
|
|
clusterResource));
|
|
clusterResource));
|
|
|
|
|
|
@@ -995,7 +993,7 @@ public class CapacityScheduler extends
|
|
|
|
|
|
|
|
|
|
allocation = application.getAllocation(getResourceCalculator(),
|
|
allocation = application.getAllocation(getResourceCalculator(),
|
|
- clusterResource, getMinimumResourceCapability());
|
|
|
|
|
|
+ getClusterResource(), getMinimumResourceCapability());
|
|
}
|
|
}
|
|
|
|
|
|
if (updateDemandForQueue != null && !application
|
|
if (updateDemandForQueue != null && !application
|
|
@@ -1036,7 +1034,8 @@ public class CapacityScheduler extends
|
|
|
|
|
|
private synchronized void nodeUpdate(RMNode nm) {
|
|
private synchronized void nodeUpdate(RMNode nm) {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
- LOG.debug("nodeUpdate: " + nm + " clusterResources: " + clusterResource);
|
|
|
|
|
|
+ LOG.debug("nodeUpdate: " + nm +
|
|
|
|
+ " clusterResources: " + getClusterResource());
|
|
}
|
|
}
|
|
|
|
|
|
Resource releaseResources = Resource.newInstance(0, 0);
|
|
Resource releaseResources = Resource.newInstance(0, 0);
|
|
@@ -1119,6 +1118,7 @@ public class CapacityScheduler extends
|
|
private synchronized void updateNodeAndQueueResource(RMNode nm,
|
|
private synchronized void updateNodeAndQueueResource(RMNode nm,
|
|
ResourceOption resourceOption) {
|
|
ResourceOption resourceOption) {
|
|
updateNodeResource(nm, resourceOption);
|
|
updateNodeResource(nm, resourceOption);
|
|
|
|
+ Resource clusterResource = getClusterResource();
|
|
root.updateClusterResource(clusterResource, new ResourceLimits(
|
|
root.updateClusterResource(clusterResource, new ResourceLimits(
|
|
clusterResource));
|
|
clusterResource));
|
|
}
|
|
}
|
|
@@ -1128,7 +1128,7 @@ public class CapacityScheduler extends
|
|
*/
|
|
*/
|
|
private synchronized void updateLabelsOnNode(NodeId nodeId,
|
|
private synchronized void updateLabelsOnNode(NodeId nodeId,
|
|
Set<String> newLabels) {
|
|
Set<String> newLabels) {
|
|
- FiCaSchedulerNode node = nodes.get(nodeId);
|
|
|
|
|
|
+ FiCaSchedulerNode node = nodeTracker.getNode(nodeId);
|
|
if (null == node) {
|
|
if (null == node) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
@@ -1230,12 +1230,12 @@ public class CapacityScheduler extends
|
|
LeafQueue queue = ((LeafQueue) reservedApplication.getQueue());
|
|
LeafQueue queue = ((LeafQueue) reservedApplication.getQueue());
|
|
assignment =
|
|
assignment =
|
|
queue.assignContainers(
|
|
queue.assignContainers(
|
|
- clusterResource,
|
|
|
|
|
|
+ getClusterResource(),
|
|
node,
|
|
node,
|
|
// TODO, now we only consider limits for parent for non-labeled
|
|
// TODO, now we only consider limits for parent for non-labeled
|
|
// resources, should consider labeled resources as well.
|
|
// resources, should consider labeled resources as well.
|
|
new ResourceLimits(labelManager.getResourceByLabel(
|
|
new ResourceLimits(labelManager.getResourceByLabel(
|
|
- RMNodeLabelsManager.NO_LABEL, clusterResource)),
|
|
|
|
|
|
+ RMNodeLabelsManager.NO_LABEL, getClusterResource())),
|
|
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
if (assignment.isFulfilledReservation()) {
|
|
if (assignment.isFulfilledReservation()) {
|
|
CSAssignment tmp =
|
|
CSAssignment tmp =
|
|
@@ -1261,14 +1261,14 @@ public class CapacityScheduler extends
|
|
}
|
|
}
|
|
|
|
|
|
assignment = root.assignContainers(
|
|
assignment = root.assignContainers(
|
|
- clusterResource,
|
|
|
|
|
|
+ getClusterResource(),
|
|
node,
|
|
node,
|
|
// TODO, now we only consider limits for parent for non-labeled
|
|
// TODO, now we only consider limits for parent for non-labeled
|
|
// resources, should consider labeled resources as well.
|
|
// resources, should consider labeled resources as well.
|
|
new ResourceLimits(labelManager.getResourceByLabel(
|
|
new ResourceLimits(labelManager.getResourceByLabel(
|
|
- RMNodeLabelsManager.NO_LABEL, clusterResource)),
|
|
|
|
|
|
+ RMNodeLabelsManager.NO_LABEL, getClusterResource())),
|
|
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
- if (Resources.greaterThan(calculator, clusterResource,
|
|
|
|
|
|
+ if (Resources.greaterThan(calculator, getClusterResource(),
|
|
assignment.getResource(), Resources.none())) {
|
|
assignment.getResource(), Resources.none())) {
|
|
updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
|
|
updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
|
|
return;
|
|
return;
|
|
@@ -1294,12 +1294,12 @@ public class CapacityScheduler extends
|
|
|
|
|
|
// Try to use NON_EXCLUSIVE
|
|
// Try to use NON_EXCLUSIVE
|
|
assignment = root.assignContainers(
|
|
assignment = root.assignContainers(
|
|
- clusterResource,
|
|
|
|
|
|
+ getClusterResource(),
|
|
node,
|
|
node,
|
|
// TODO, now we only consider limits for parent for non-labeled
|
|
// TODO, now we only consider limits for parent for non-labeled
|
|
// resources, should consider labeled resources as well.
|
|
// resources, should consider labeled resources as well.
|
|
new ResourceLimits(labelManager.getResourceByLabel(
|
|
new ResourceLimits(labelManager.getResourceByLabel(
|
|
- RMNodeLabelsManager.NO_LABEL, clusterResource)),
|
|
|
|
|
|
+ RMNodeLabelsManager.NO_LABEL, getClusterResource())),
|
|
SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY);
|
|
SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY);
|
|
updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
|
|
updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
|
|
}
|
|
}
|
|
@@ -1451,24 +1451,22 @@ public class CapacityScheduler extends
|
|
private synchronized void addNode(RMNode nodeManager) {
|
|
private synchronized void addNode(RMNode nodeManager) {
|
|
FiCaSchedulerNode schedulerNode = new FiCaSchedulerNode(nodeManager,
|
|
FiCaSchedulerNode schedulerNode = new FiCaSchedulerNode(nodeManager,
|
|
usePortForNodeName, nodeManager.getNodeLabels());
|
|
usePortForNodeName, nodeManager.getNodeLabels());
|
|
- this.nodes.put(nodeManager.getNodeID(), schedulerNode);
|
|
|
|
- Resources.addTo(clusterResource, schedulerNode.getTotalResource());
|
|
|
|
|
|
+ nodeTracker.addNode(schedulerNode);
|
|
|
|
|
|
// update this node to node label manager
|
|
// update this node to node label manager
|
|
if (labelManager != null) {
|
|
if (labelManager != null) {
|
|
labelManager.activateNode(nodeManager.getNodeID(),
|
|
labelManager.activateNode(nodeManager.getNodeID(),
|
|
schedulerNode.getTotalResource());
|
|
schedulerNode.getTotalResource());
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ Resource clusterResource = getClusterResource();
|
|
root.updateClusterResource(clusterResource, new ResourceLimits(
|
|
root.updateClusterResource(clusterResource, new ResourceLimits(
|
|
clusterResource));
|
|
clusterResource));
|
|
- int numNodes = numNodeManagers.incrementAndGet();
|
|
|
|
- updateMaximumAllocation(schedulerNode, true);
|
|
|
|
-
|
|
|
|
|
|
+
|
|
LOG.info("Added node " + nodeManager.getNodeAddress() +
|
|
LOG.info("Added node " + nodeManager.getNodeAddress() +
|
|
" clusterResource: " + clusterResource);
|
|
" clusterResource: " + clusterResource);
|
|
|
|
|
|
- if (scheduleAsynchronously && numNodes == 1) {
|
|
|
|
|
|
+ if (scheduleAsynchronously && getNumClusterNodes() == 1) {
|
|
asyncSchedulerThread.beginSchedule();
|
|
asyncSchedulerThread.beginSchedule();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -1478,20 +1476,14 @@ public class CapacityScheduler extends
|
|
if (labelManager != null) {
|
|
if (labelManager != null) {
|
|
labelManager.deactivateNode(nodeInfo.getNodeID());
|
|
labelManager.deactivateNode(nodeInfo.getNodeID());
|
|
}
|
|
}
|
|
-
|
|
|
|
- FiCaSchedulerNode node = nodes.get(nodeInfo.getNodeID());
|
|
|
|
|
|
+
|
|
|
|
+ NodeId nodeId = nodeInfo.getNodeID();
|
|
|
|
+ FiCaSchedulerNode node = nodeTracker.getNode(nodeId);
|
|
if (node == null) {
|
|
if (node == null) {
|
|
|
|
+ LOG.error("Attempting to remove non-existent node " + nodeId);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
- Resources.subtractFrom(clusterResource, node.getTotalResource());
|
|
|
|
- root.updateClusterResource(clusterResource, new ResourceLimits(
|
|
|
|
- clusterResource));
|
|
|
|
- int numNodes = numNodeManagers.decrementAndGet();
|
|
|
|
|
|
|
|
- if (scheduleAsynchronously && numNodes == 0) {
|
|
|
|
- asyncSchedulerThread.suspendSchedule();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
// Remove running containers
|
|
// Remove running containers
|
|
List<RMContainer> runningContainers = node.getRunningContainers();
|
|
List<RMContainer> runningContainers = node.getRunningContainers();
|
|
for (RMContainer container : runningContainers) {
|
|
for (RMContainer container : runningContainers) {
|
|
@@ -1512,11 +1504,18 @@ public class CapacityScheduler extends
|
|
RMContainerEventType.KILL);
|
|
RMContainerEventType.KILL);
|
|
}
|
|
}
|
|
|
|
|
|
- this.nodes.remove(nodeInfo.getNodeID());
|
|
|
|
- updateMaximumAllocation(node, false);
|
|
|
|
|
|
+ nodeTracker.removeNode(nodeId);
|
|
|
|
+ Resource clusterResource = getClusterResource();
|
|
|
|
+ root.updateClusterResource(clusterResource, new ResourceLimits(
|
|
|
|
+ clusterResource));
|
|
|
|
+ int numNodes = nodeTracker.nodeCount();
|
|
|
|
+
|
|
|
|
+ if (scheduleAsynchronously && numNodes == 0) {
|
|
|
|
+ asyncSchedulerThread.suspendSchedule();
|
|
|
|
+ }
|
|
|
|
|
|
LOG.info("Removed node " + nodeInfo.getNodeAddress() +
|
|
LOG.info("Removed node " + nodeInfo.getNodeAddress() +
|
|
- " clusterResource: " + clusterResource);
|
|
|
|
|
|
+ " clusterResource: " + getClusterResource());
|
|
}
|
|
}
|
|
|
|
|
|
private void rollbackContainerResource(
|
|
private void rollbackContainerResource(
|
|
@@ -1568,7 +1567,7 @@ public class CapacityScheduler extends
|
|
|
|
|
|
// Inform the queue
|
|
// Inform the queue
|
|
LeafQueue queue = (LeafQueue)application.getQueue();
|
|
LeafQueue queue = (LeafQueue)application.getQueue();
|
|
- queue.completedContainer(clusterResource, application, node,
|
|
|
|
|
|
+ queue.completedContainer(getClusterResource(), application, node,
|
|
rmContainer, containerStatus, event, null, true);
|
|
rmContainer, containerStatus, event, null, true);
|
|
|
|
|
|
if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
|
|
if (containerStatus.getExitStatus() == ContainerExitStatus.PREEMPTED) {
|
|
@@ -1594,7 +1593,7 @@ public class CapacityScheduler extends
|
|
FiCaSchedulerApp app = (FiCaSchedulerApp)attempt;
|
|
FiCaSchedulerApp app = (FiCaSchedulerApp)attempt;
|
|
LeafQueue queue = (LeafQueue) attempt.getQueue();
|
|
LeafQueue queue = (LeafQueue) attempt.getQueue();
|
|
try {
|
|
try {
|
|
- queue.decreaseContainer(clusterResource, decreaseRequest, app);
|
|
|
|
|
|
+ queue.decreaseContainer(getClusterResource(), decreaseRequest, app);
|
|
// Notify RMNode that the container can be pulled by NodeManager in the
|
|
// Notify RMNode that the container can be pulled by NodeManager in the
|
|
// next heartbeat
|
|
// next heartbeat
|
|
this.rmContext.getDispatcher().getEventHandler()
|
|
this.rmContext.getDispatcher().getEventHandler()
|
|
@@ -1617,14 +1616,9 @@ public class CapacityScheduler extends
|
|
|
|
|
|
@Lock(Lock.NoLock.class)
|
|
@Lock(Lock.NoLock.class)
|
|
public FiCaSchedulerNode getNode(NodeId nodeId) {
|
|
public FiCaSchedulerNode getNode(NodeId nodeId) {
|
|
- return nodes.get(nodeId);
|
|
|
|
|
|
+ return nodeTracker.getNode(nodeId);
|
|
}
|
|
}
|
|
|
|
|
|
- @Lock(Lock.NoLock.class)
|
|
|
|
- Map<NodeId, FiCaSchedulerNode> getAllNodes() {
|
|
|
|
- return nodes;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
@Override
|
|
@Override
|
|
@Lock(Lock.NoLock.class)
|
|
@Lock(Lock.NoLock.class)
|
|
public void recover(RMState state) throws Exception {
|
|
public void recover(RMState state) throws Exception {
|
|
@@ -1869,9 +1863,9 @@ public class CapacityScheduler extends
|
|
}
|
|
}
|
|
// Move all live containers
|
|
// Move all live containers
|
|
for (RMContainer rmContainer : app.getLiveContainers()) {
|
|
for (RMContainer rmContainer : app.getLiveContainers()) {
|
|
- source.detachContainer(clusterResource, app, rmContainer);
|
|
|
|
|
|
+ source.detachContainer(getClusterResource(), app, rmContainer);
|
|
// attach the Container to another queue
|
|
// attach the Container to another queue
|
|
- dest.attachContainer(clusterResource, app, rmContainer);
|
|
|
|
|
|
+ dest.attachContainer(getClusterResource(), app, rmContainer);
|
|
}
|
|
}
|
|
// Detach the application..
|
|
// Detach the application..
|
|
source.finishApplicationAttempt(app, sourceQueueName);
|
|
source.finishApplicationAttempt(app, sourceQueueName);
|