|
@@ -69,6 +69,7 @@ import org.apache.hadoop.yarn.api.records.ResourceOption;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceSizing;
|
|
|
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
|
|
|
+import org.apache.hadoop.yarn.api.records.NodeState;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
@@ -232,7 +233,7 @@ public class CapacityScheduler extends
|
|
|
|
|
|
private CapacitySchedulerAutoQueueHandler autoQueueHandler;
|
|
|
|
|
|
- private static boolean printedVerboseLoggingForAsyncScheduling = false;
|
|
|
+ private boolean printedVerboseLoggingForAsyncScheduling;
|
|
|
|
|
|
/**
|
|
|
* EXPERT
|
|
@@ -518,22 +519,47 @@ public class CapacityScheduler extends
|
|
|
|
|
|
private final static Random random = new Random(System.currentTimeMillis());
|
|
|
|
|
|
- private static boolean shouldSkipNodeSchedule(FiCaSchedulerNode node,
|
|
|
+ @VisibleForTesting
|
|
|
+ public static boolean shouldSkipNodeSchedule(FiCaSchedulerNode node,
|
|
|
CapacityScheduler cs, boolean printVerboseLog) {
|
|
|
- // Skip node which missed 2 heartbeats since the node might be dead and
|
|
|
- // we should not continue allocate containers on that.
|
|
|
- long timeElapsedFromLastHeartbeat =
|
|
|
- Time.monotonicNow() - node.getLastHeartbeatMonotonicTime();
|
|
|
- if (timeElapsedFromLastHeartbeat > cs.nmHeartbeatInterval * 2) {
|
|
|
+ // Skip node which missed YarnConfiguration.SCHEDULER_SKIP_NODE_MULTIPLIER
|
|
|
+ // heartbeats since the node might be dead and we should not continue
|
|
|
+ // allocate containers on that.
|
|
|
+ if (!SchedulerUtils.isNodeHeartbeated(node, cs.getSkipNodeInterval())) {
|
|
|
if (printVerboseLog && LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Skip scheduling on node because it haven't heartbeated for "
|
|
|
+ long timeElapsedFromLastHeartbeat =
|
|
|
+ Time.monotonicNow() - node.getLastHeartbeatMonotonicTime();
|
|
|
+ LOG.debug("Skip scheduling on node " + node.getNodeID()
|
|
|
+ + " because it haven't heartbeated for "
|
|
|
+ timeElapsedFromLastHeartbeat / 1000.0f + " secs");
|
|
|
}
|
|
|
return true;
|
|
|
}
|
|
|
+
|
|
|
+ if (node.getRMNode().getState() != NodeState.RUNNING) {
|
|
|
+ if (printVerboseLog && LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Skip scheduling on node because it is in " +
|
|
|
+ node.getRMNode().getState() + " state");
|
|
|
+ }
|
|
|
+ return true;
|
|
|
+ }
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
+ private static boolean isPrintSkippedNodeLogging(CapacityScheduler cs) {
|
|
|
+ // To avoid too verbose DEBUG logging, only print debug log once for
|
|
|
+ // every 10 secs.
|
|
|
+ boolean printSkipedNodeLogging = false;
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ if (Time.monotonicNow() / 1000 % 10 == 0) {
|
|
|
+ printSkipedNodeLogging = (!cs.printedVerboseLoggingForAsyncScheduling);
|
|
|
+ } else {
|
|
|
+ cs.printedVerboseLoggingForAsyncScheduling = false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return printSkipedNodeLogging;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Schedule on all nodes by starting at a random point.
|
|
|
* Schedule on all partitions by starting at a random partition
|
|
@@ -555,19 +581,12 @@ public class CapacityScheduler extends
|
|
|
if (!cs.multiNodePlacementEnabled) {
|
|
|
int start = random.nextInt(nodeSize);
|
|
|
|
|
|
- // To avoid too verbose DEBUG logging, only print debug log once for
|
|
|
- // every 10 secs.
|
|
|
- boolean printSkipedNodeLogging = false;
|
|
|
- if (Time.monotonicNow() / 1000 % 10 == 0) {
|
|
|
- printSkipedNodeLogging = (!printedVerboseLoggingForAsyncScheduling);
|
|
|
- } else {
|
|
|
- printedVerboseLoggingForAsyncScheduling = false;
|
|
|
- }
|
|
|
+ boolean printSkippedNodeLogging = isPrintSkippedNodeLogging(cs);
|
|
|
|
|
|
// Allocate containers of node [start, end)
|
|
|
for (FiCaSchedulerNode node : nodes) {
|
|
|
if (current++ >= start) {
|
|
|
- if (shouldSkipNodeSchedule(node, cs, printSkipedNodeLogging)) {
|
|
|
+ if (shouldSkipNodeSchedule(node, cs, printSkippedNodeLogging)) {
|
|
|
continue;
|
|
|
}
|
|
|
cs.allocateContainersToNode(node.getNodeID(), false);
|
|
@@ -581,14 +600,14 @@ public class CapacityScheduler extends
|
|
|
if (current++ > start) {
|
|
|
break;
|
|
|
}
|
|
|
- if (shouldSkipNodeSchedule(node, cs, printSkipedNodeLogging)) {
|
|
|
+ if (shouldSkipNodeSchedule(node, cs, printSkippedNodeLogging)) {
|
|
|
continue;
|
|
|
}
|
|
|
cs.allocateContainersToNode(node.getNodeID(), false);
|
|
|
}
|
|
|
|
|
|
- if (printSkipedNodeLogging) {
|
|
|
- printedVerboseLoggingForAsyncScheduling = true;
|
|
|
+ if (printSkippedNodeLogging) {
|
|
|
+ cs.printedVerboseLoggingForAsyncScheduling = true;
|
|
|
}
|
|
|
} else {
|
|
|
// Get all partitions
|
|
@@ -1541,20 +1560,37 @@ public class CapacityScheduler extends
|
|
|
|| assignedContainers < maxAssignPerHeartbeat);
|
|
|
}
|
|
|
|
|
|
- private CandidateNodeSet<FiCaSchedulerNode> getCandidateNodeSet(
|
|
|
- String partition) {
|
|
|
- CandidateNodeSet<FiCaSchedulerNode> candidates = null;
|
|
|
+ private Map<NodeId, FiCaSchedulerNode> getNodesHeartbeated(String partition) {
|
|
|
Map<NodeId, FiCaSchedulerNode> nodesByPartition = new HashMap<>();
|
|
|
+ boolean printSkippedNodeLogging = isPrintSkippedNodeLogging(this);
|
|
|
List<FiCaSchedulerNode> nodes = nodeTracker
|
|
|
- .getNodesPerPartition(partition);
|
|
|
+ .getNodesPerPartition(partition);
|
|
|
+
|
|
|
if (nodes != null && !nodes.isEmpty()) {
|
|
|
//Filter for node heartbeat too long
|
|
|
nodes.stream()
|
|
|
- .filter(node -> !shouldSkipNodeSchedule(node, this, true))
|
|
|
- .forEach(n -> nodesByPartition.put(n.getNodeID(), n));
|
|
|
+ .filter(node ->
|
|
|
+ !shouldSkipNodeSchedule(node, this, printSkippedNodeLogging))
|
|
|
+ .forEach(n -> nodesByPartition.put(n.getNodeID(), n));
|
|
|
+ }
|
|
|
+
|
|
|
+ if (printSkippedNodeLogging) {
|
|
|
+ printedVerboseLoggingForAsyncScheduling = true;
|
|
|
+ }
|
|
|
+ return nodesByPartition;
|
|
|
+ }
|
|
|
+
|
|
|
+ private CandidateNodeSet<FiCaSchedulerNode> getCandidateNodeSet(
|
|
|
+ String partition) {
|
|
|
+ CandidateNodeSet<FiCaSchedulerNode> candidates = null;
|
|
|
+ Map<NodeId, FiCaSchedulerNode> nodesByPartition
|
|
|
+ = getNodesHeartbeated(partition);
|
|
|
+
|
|
|
+ if (!nodesByPartition.isEmpty()) {
|
|
|
candidates = new SimpleCandidateNodeSet<FiCaSchedulerNode>(
|
|
|
- nodesByPartition, partition);
|
|
|
+ nodesByPartition, partition);
|
|
|
}
|
|
|
+
|
|
|
return candidates;
|
|
|
}
|
|
|
|
|
@@ -1563,11 +1599,9 @@ public class CapacityScheduler extends
|
|
|
CandidateNodeSet<FiCaSchedulerNode> candidates = null;
|
|
|
candidates = new SimpleCandidateNodeSet<>(node);
|
|
|
if (multiNodePlacementEnabled) {
|
|
|
- Map<NodeId, FiCaSchedulerNode> nodesByPartition = new HashMap<>();
|
|
|
- List<FiCaSchedulerNode> nodes = nodeTracker
|
|
|
- .getNodesPerPartition(node.getPartition());
|
|
|
- if (nodes != null && !nodes.isEmpty()) {
|
|
|
- nodes.forEach(n -> nodesByPartition.put(n.getNodeID(), n));
|
|
|
+ Map<NodeId, FiCaSchedulerNode> nodesByPartition =
|
|
|
+ getNodesHeartbeated(node.getPartition());
|
|
|
+ if (!nodesByPartition.isEmpty()) {
|
|
|
candidates = new SimpleCandidateNodeSet<FiCaSchedulerNode>(
|
|
|
nodesByPartition, node.getPartition());
|
|
|
}
|