YARN-10399 Refactor NodeQueueLoadMonitor class to make it extendable (#2228)
Refactor NodeQueueLoadMonitor class to make it extendable
This commit is contained in:
parent
dd013f2fdf
commit
9b9f7ea16a
@ -0,0 +1,93 @@
|
|||||||
|
/**
|
||||||
|
* 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.hadoop.yarn.server.resourcemanager.scheduler.distributed;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Represents a node in the cluster from the NodeQueueLoadMonitor's perspective
|
||||||
|
*/
|
||||||
|
public class ClusterNode {
|
||||||
|
private final AtomicInteger queueLength = new AtomicInteger(0);
|
||||||
|
private final AtomicInteger queueWaitTime = new AtomicInteger(-1);
|
||||||
|
private long timestamp;
|
||||||
|
final NodeId nodeId;
|
||||||
|
private int queueCapacity = 0;
|
||||||
|
private final HashSet<String> labels;
|
||||||
|
|
||||||
|
public ClusterNode(NodeId nodeId) {
|
||||||
|
this.nodeId = nodeId;
|
||||||
|
this.labels = new HashSet<>();
|
||||||
|
updateTimestamp();
|
||||||
|
}
|
||||||
|
|
||||||
|
public ClusterNode setQueueLength(int qLength) {
|
||||||
|
this.queueLength.set(qLength);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ClusterNode setQueueWaitTime(int wTime) {
|
||||||
|
this.queueWaitTime.set(wTime);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ClusterNode updateTimestamp() {
|
||||||
|
this.timestamp = System.currentTimeMillis();
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ClusterNode setQueueCapacity(int capacity) {
|
||||||
|
this.queueCapacity = capacity;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ClusterNode setNodeLabels(Collection<String> labelsToAdd) {
|
||||||
|
labels.clear();
|
||||||
|
labels.addAll(labelsToAdd);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean hasLabel(String label) {
|
||||||
|
return this.labels.contains(label);
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getTimestamp() {
|
||||||
|
return this.timestamp;
|
||||||
|
}
|
||||||
|
|
||||||
|
public AtomicInteger getQueueLength() {
|
||||||
|
return this.queueLength;
|
||||||
|
}
|
||||||
|
|
||||||
|
public AtomicInteger getQueueWaitTime() {
|
||||||
|
return this.queueWaitTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getQueueCapacity() {
|
||||||
|
return this.queueCapacity;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isQueueFull() {
|
||||||
|
return this.queueCapacity > 0 &&
|
||||||
|
this.queueLength.get() >= this.queueCapacity;
|
||||||
|
}
|
||||||
|
}
|
@ -40,8 +40,8 @@
|
|||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED;
|
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED;
|
||||||
|
|
||||||
@ -53,10 +53,10 @@
|
|||||||
*/
|
*/
|
||||||
public class NodeQueueLoadMonitor implements ClusterMonitor {
|
public class NodeQueueLoadMonitor implements ClusterMonitor {
|
||||||
|
|
||||||
private final static Logger LOG = LoggerFactory.
|
protected final static Logger LOG = LoggerFactory.
|
||||||
getLogger(NodeQueueLoadMonitor.class);
|
getLogger(NodeQueueLoadMonitor.class);
|
||||||
|
|
||||||
private int numNodesForAnyAllocation =
|
protected int numNodesForAnyAllocation =
|
||||||
DEFAULT_OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED;
|
DEFAULT_OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -70,14 +70,14 @@ public enum LoadComparator implements Comparator<ClusterNode> {
|
|||||||
@Override
|
@Override
|
||||||
public int compare(ClusterNode o1, ClusterNode o2) {
|
public int compare(ClusterNode o1, ClusterNode o2) {
|
||||||
if (getMetric(o1) == getMetric(o2)) {
|
if (getMetric(o1) == getMetric(o2)) {
|
||||||
return (int)(o2.timestamp - o1.timestamp);
|
return (int)(o2.getTimestamp() - o1.getTimestamp());
|
||||||
}
|
}
|
||||||
return getMetric(o1) - getMetric(o2);
|
return getMetric(o1) - getMetric(o2);
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getMetric(ClusterNode c) {
|
public int getMetric(ClusterNode c) {
|
||||||
return (this == QUEUE_LENGTH) ?
|
return (this == QUEUE_LENGTH) ?
|
||||||
c.queueLength.get() : c.queueWaitTime.get();
|
c.getQueueLength().get() : c.getQueueWaitTime().get();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -88,11 +88,11 @@ public int getMetric(ClusterNode c) {
|
|||||||
*/
|
*/
|
||||||
public boolean compareAndIncrement(ClusterNode c, int incrementSize) {
|
public boolean compareAndIncrement(ClusterNode c, int incrementSize) {
|
||||||
if(this == QUEUE_LENGTH) {
|
if(this == QUEUE_LENGTH) {
|
||||||
int ret = c.queueLength.addAndGet(incrementSize);
|
int ret = c.getQueueLength().addAndGet(incrementSize);
|
||||||
if (ret <= c.queueCapacity) {
|
if (ret <= c.getQueueCapacity()) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
c.queueLength.addAndGet(-incrementSize);
|
c.getQueueLength().addAndGet(-incrementSize);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
// for queue wait time, we don't have any threshold.
|
// for queue wait time, we don't have any threshold.
|
||||||
@ -100,57 +100,19 @@ public boolean compareAndIncrement(ClusterNode c, int incrementSize) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static class ClusterNode {
|
|
||||||
private AtomicInteger queueLength = new AtomicInteger(0);
|
|
||||||
private AtomicInteger queueWaitTime = new AtomicInteger(-1);
|
|
||||||
private long timestamp;
|
|
||||||
final NodeId nodeId;
|
|
||||||
private int queueCapacity = 0;
|
|
||||||
|
|
||||||
public ClusterNode(NodeId nodeId) {
|
|
||||||
this.nodeId = nodeId;
|
|
||||||
updateTimestamp();
|
|
||||||
}
|
|
||||||
|
|
||||||
public ClusterNode setQueueLength(int qLength) {
|
|
||||||
this.queueLength.set(qLength);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ClusterNode setQueueWaitTime(int wTime) {
|
|
||||||
this.queueWaitTime.set(wTime);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ClusterNode updateTimestamp() {
|
|
||||||
this.timestamp = System.currentTimeMillis();
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ClusterNode setQueueCapacity(int capacity) {
|
|
||||||
this.queueCapacity = capacity;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean isQueueFull() {
|
|
||||||
return this.queueCapacity > 0 &&
|
|
||||||
this.queueLength.get() >= this.queueCapacity;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private final ScheduledExecutorService scheduledExecutor;
|
private final ScheduledExecutorService scheduledExecutor;
|
||||||
|
|
||||||
private final List<NodeId> sortedNodes;
|
protected final List<NodeId> sortedNodes;
|
||||||
private final Map<NodeId, ClusterNode> clusterNodes =
|
protected final Map<NodeId, ClusterNode> clusterNodes =
|
||||||
new ConcurrentHashMap<>();
|
new ConcurrentHashMap<>();
|
||||||
private final Map<String, RMNode> nodeByHostName =
|
protected final Map<String, RMNode> nodeByHostName =
|
||||||
new ConcurrentHashMap<>();
|
new ConcurrentHashMap<>();
|
||||||
private final Map<String, Set<NodeId>> nodeIdsByRack =
|
protected final Map<String, Set<NodeId>> nodeIdsByRack =
|
||||||
new ConcurrentHashMap<>();
|
new ConcurrentHashMap<>();
|
||||||
private final LoadComparator comparator;
|
protected final LoadComparator comparator;
|
||||||
private QueueLimitCalculator thresholdCalculator;
|
protected QueueLimitCalculator thresholdCalculator;
|
||||||
private ReentrantReadWriteLock sortedNodesLock = new ReentrantReadWriteLock();
|
protected ReentrantReadWriteLock sortedNodesLock = new ReentrantReadWriteLock();
|
||||||
private ReentrantReadWriteLock clusterNodesLock =
|
protected ReentrantReadWriteLock clusterNodesLock =
|
||||||
new ReentrantReadWriteLock();
|
new ReentrantReadWriteLock();
|
||||||
|
|
||||||
Runnable computeTask = new Runnable() {
|
Runnable computeTask = new Runnable() {
|
||||||
@ -160,9 +122,7 @@ public void run() {
|
|||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
try {
|
try {
|
||||||
try {
|
try {
|
||||||
List<NodeId> nodeIds = sortNodes();
|
updateSortedNodes();
|
||||||
sortedNodes.clear();
|
|
||||||
sortedNodes.addAll(nodeIds);
|
|
||||||
} catch (Exception ex) {
|
} catch (Exception ex) {
|
||||||
LOG.warn("Got Exception while sorting nodes..", ex);
|
LOG.warn("Got Exception while sorting nodes..", ex);
|
||||||
}
|
}
|
||||||
@ -193,6 +153,14 @@ public NodeQueueLoadMonitor(long nodeComputationInterval,
|
|||||||
numNodesForAnyAllocation = numNodes;
|
numNodesForAnyAllocation = numNodes;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected void updateSortedNodes() {
|
||||||
|
List<NodeId> nodeIds = sortNodes(true).stream()
|
||||||
|
.map(n -> n.nodeId)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
sortedNodes.clear();
|
||||||
|
sortedNodes.addAll(nodeIds);
|
||||||
|
}
|
||||||
|
|
||||||
List<NodeId> getSortedNodes() {
|
List<NodeId> getSortedNodes() {
|
||||||
return sortedNodes;
|
return sortedNodes;
|
||||||
}
|
}
|
||||||
@ -239,6 +207,7 @@ public void removeNode(RMNode removedRMNode) {
|
|||||||
ClusterNode node;
|
ClusterNode node;
|
||||||
try {
|
try {
|
||||||
node = this.clusterNodes.remove(removedRMNode.getNodeID());
|
node = this.clusterNodes.remove(removedRMNode.getNodeID());
|
||||||
|
onNodeRemoved(node);
|
||||||
} finally {
|
} finally {
|
||||||
writeLock.unlock();
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
@ -251,6 +220,13 @@ public void removeNode(RMNode removedRMNode) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Provide an integration point for extended class
|
||||||
|
* @param node the node removed
|
||||||
|
*/
|
||||||
|
protected void onNodeRemoved(ClusterNode node) {
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void updateNode(RMNode rmNode) {
|
public void updateNode(RMNode rmNode) {
|
||||||
LOG.debug("Node update event from: {}", rmNode.getNodeID());
|
LOG.debug("Node update event from: {}", rmNode.getNodeID());
|
||||||
@ -260,58 +236,83 @@ public void updateNode(RMNode rmNode) {
|
|||||||
opportunisticContainersStatus =
|
opportunisticContainersStatus =
|
||||||
OpportunisticContainersStatus.newInstance();
|
OpportunisticContainersStatus.newInstance();
|
||||||
}
|
}
|
||||||
int opportQueueCapacity =
|
|
||||||
opportunisticContainersStatus.getOpportQueueCapacity();
|
|
||||||
int estimatedQueueWaitTime =
|
|
||||||
opportunisticContainersStatus.getEstimatedQueueWaitTime();
|
|
||||||
int waitQueueLength = opportunisticContainersStatus.getWaitQueueLength();
|
|
||||||
// Add nodes to clusterNodes. If estimatedQueueTime is -1, ignore node
|
// Add nodes to clusterNodes. If estimatedQueueTime is -1, ignore node
|
||||||
// UNLESS comparator is based on queue length.
|
// UNLESS comparator is based on queue length.
|
||||||
ReentrantReadWriteLock.WriteLock writeLock = clusterNodesLock.writeLock();
|
ReentrantReadWriteLock.WriteLock writeLock = clusterNodesLock.writeLock();
|
||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
try {
|
try {
|
||||||
ClusterNode currentNode = this.clusterNodes.get(rmNode.getNodeID());
|
ClusterNode clusterNode = this.clusterNodes.get(rmNode.getNodeID());
|
||||||
if (currentNode == null) {
|
if (clusterNode == null) {
|
||||||
if (rmNode.getState() != NodeState.DECOMMISSIONING &&
|
onNewNodeAdded(rmNode, opportunisticContainersStatus);
|
||||||
(estimatedQueueWaitTime != -1 ||
|
|
||||||
comparator == LoadComparator.QUEUE_LENGTH)) {
|
|
||||||
this.clusterNodes.put(rmNode.getNodeID(),
|
|
||||||
new ClusterNode(rmNode.getNodeID())
|
|
||||||
.setQueueWaitTime(estimatedQueueWaitTime)
|
|
||||||
.setQueueLength(waitQueueLength)
|
|
||||||
.setQueueCapacity(opportQueueCapacity));
|
|
||||||
LOG.info("Inserting ClusterNode [" + rmNode.getNodeID() + "] " +
|
|
||||||
"with queue wait time [" + estimatedQueueWaitTime + "] and " +
|
|
||||||
"wait queue length [" + waitQueueLength + "]");
|
|
||||||
} else {
|
|
||||||
LOG.warn("IGNORING ClusterNode [" + rmNode.getNodeID() + "] " +
|
|
||||||
"with queue wait time [" + estimatedQueueWaitTime + "] and " +
|
|
||||||
"wait queue length [" + waitQueueLength + "]");
|
|
||||||
}
|
|
||||||
} else {
|
} else {
|
||||||
if (rmNode.getState() != NodeState.DECOMMISSIONING &&
|
onExistingNodeUpdated(rmNode, clusterNode, opportunisticContainersStatus);
|
||||||
(estimatedQueueWaitTime != -1 ||
|
|
||||||
comparator == LoadComparator.QUEUE_LENGTH)) {
|
|
||||||
currentNode
|
|
||||||
.setQueueWaitTime(estimatedQueueWaitTime)
|
|
||||||
.setQueueLength(waitQueueLength)
|
|
||||||
.updateTimestamp();
|
|
||||||
LOG.debug("Updating ClusterNode [{}] with queue wait time [{}] and"
|
|
||||||
+ " wait queue length [{}]", rmNode.getNodeID(),
|
|
||||||
estimatedQueueWaitTime, waitQueueLength);
|
|
||||||
|
|
||||||
} else {
|
|
||||||
this.clusterNodes.remove(rmNode.getNodeID());
|
|
||||||
LOG.info("Deleting ClusterNode [" + rmNode.getNodeID() + "] " +
|
|
||||||
"with queue wait time [" + currentNode.queueWaitTime + "] and " +
|
|
||||||
"wait queue length [" + currentNode.queueLength + "]");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
writeLock.unlock();
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected void onNewNodeAdded(
|
||||||
|
RMNode rmNode, OpportunisticContainersStatus status) {
|
||||||
|
int opportQueueCapacity = status.getOpportQueueCapacity();
|
||||||
|
int estimatedQueueWaitTime = status.getEstimatedQueueWaitTime();
|
||||||
|
int waitQueueLength = status.getWaitQueueLength();
|
||||||
|
|
||||||
|
if (rmNode.getState() != NodeState.DECOMMISSIONING &&
|
||||||
|
(estimatedQueueWaitTime != -1 ||
|
||||||
|
comparator == LoadComparator.QUEUE_LENGTH)) {
|
||||||
|
this.clusterNodes.put(rmNode.getNodeID(),
|
||||||
|
new ClusterNode(rmNode.getNodeID())
|
||||||
|
.setQueueWaitTime(estimatedQueueWaitTime)
|
||||||
|
.setQueueLength(waitQueueLength)
|
||||||
|
.setNodeLabels(rmNode.getNodeLabels())
|
||||||
|
.setQueueCapacity(opportQueueCapacity));
|
||||||
|
LOG.info(
|
||||||
|
"Inserting ClusterNode [{}] with queue wait time [{}] and "
|
||||||
|
+ "wait queue length [{}]",
|
||||||
|
rmNode.getNode(),
|
||||||
|
estimatedQueueWaitTime,
|
||||||
|
waitQueueLength
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
LOG.warn(
|
||||||
|
"IGNORING ClusterNode [{}] with queue wait time [{}] and "
|
||||||
|
+ "wait queue length [{}]",
|
||||||
|
rmNode.getNode(),
|
||||||
|
estimatedQueueWaitTime,
|
||||||
|
waitQueueLength
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void onExistingNodeUpdated(
|
||||||
|
RMNode rmNode, ClusterNode clusterNode,
|
||||||
|
OpportunisticContainersStatus status) {
|
||||||
|
|
||||||
|
int estimatedQueueWaitTime = status.getEstimatedQueueWaitTime();
|
||||||
|
int waitQueueLength = status.getWaitQueueLength();
|
||||||
|
|
||||||
|
if (rmNode.getState() != NodeState.DECOMMISSIONING &&
|
||||||
|
(estimatedQueueWaitTime != -1 ||
|
||||||
|
comparator == LoadComparator.QUEUE_LENGTH)) {
|
||||||
|
clusterNode
|
||||||
|
.setQueueWaitTime(estimatedQueueWaitTime)
|
||||||
|
.setQueueLength(waitQueueLength)
|
||||||
|
.setNodeLabels(rmNode.getNodeLabels())
|
||||||
|
.updateTimestamp();
|
||||||
|
LOG.debug("Updating ClusterNode [{}] with queue wait time [{}] and"
|
||||||
|
+ " wait queue length [{}]", rmNode.getNodeID(),
|
||||||
|
estimatedQueueWaitTime, waitQueueLength);
|
||||||
|
|
||||||
|
} else {
|
||||||
|
this.clusterNodes.remove(rmNode.getNodeID());
|
||||||
|
LOG.info("Deleting ClusterNode [" + rmNode.getNodeID() + "] " +
|
||||||
|
"with queue wait time [" + clusterNode.getQueueWaitTime() + "] and " +
|
||||||
|
"wait queue length [" + clusterNode.getQueueLength() + "]");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void updateNodeResource(RMNode rmNode, ResourceOption resourceOption) {
|
public void updateNodeResource(RMNode rmNode, ResourceOption resourceOption) {
|
||||||
LOG.debug("Node resource update event from: {}", rmNode.getNodeID());
|
LOG.debug("Node resource update event from: {}", rmNode.getNodeID());
|
||||||
@ -374,7 +375,7 @@ public RMNode selectRackLocalNode(String rackName, Set<String> blacklist) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public RMNode selectAnyNode(Set<String> blacklist) {
|
public RMNode selectAnyNode(Set<String> blacklist) {
|
||||||
List<NodeId> nodeIds = selectLeastLoadedNodes(numNodesForAnyAllocation);
|
List<NodeId> nodeIds = getCandidatesForSelectAnyNode();
|
||||||
int size = nodeIds.size();
|
int size = nodeIds.size();
|
||||||
if (size <= 0) {
|
if (size <= 0) {
|
||||||
return null;
|
return null;
|
||||||
@ -395,22 +396,26 @@ public RMNode selectAnyNode(Set<String> blacklist) {
|
|||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void removeFromNodeIdsByRack(RMNode removedNode) {
|
protected List<NodeId> getCandidatesForSelectAnyNode() {
|
||||||
|
return selectLeastLoadedNodes(numNodesForAnyAllocation);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void removeFromNodeIdsByRack(RMNode removedNode) {
|
||||||
nodeIdsByRack.computeIfPresent(removedNode.getRackName(),
|
nodeIdsByRack.computeIfPresent(removedNode.getRackName(),
|
||||||
(k, v) -> v).remove(removedNode.getNodeID());
|
(k, v) -> v).remove(removedNode.getNodeID());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void addIntoNodeIdsByRack(RMNode addedNode) {
|
protected void addIntoNodeIdsByRack(RMNode addedNode) {
|
||||||
nodeIdsByRack.compute(addedNode.getRackName(), (k, v) -> v == null ?
|
nodeIdsByRack.compute(addedNode.getRackName(), (k, v) -> v == null ?
|
||||||
ConcurrentHashMap.newKeySet() : v).add(addedNode.getNodeID());
|
ConcurrentHashMap.newKeySet() : v).add(addedNode.getNodeID());
|
||||||
}
|
}
|
||||||
|
|
||||||
private List<NodeId> sortNodes() {
|
protected List<ClusterNode> sortNodes(boolean excludeFullNodes) {
|
||||||
ReentrantReadWriteLock.ReadLock readLock = clusterNodesLock.readLock();
|
ReentrantReadWriteLock.ReadLock readLock = clusterNodesLock.readLock();
|
||||||
readLock.lock();
|
readLock.lock();
|
||||||
try {
|
try {
|
||||||
ArrayList aList = new ArrayList<>(this.clusterNodes.values());
|
ArrayList<ClusterNode> aList = new ArrayList<>(this.clusterNodes.values());
|
||||||
List<NodeId> retList = new ArrayList<>();
|
List<ClusterNode> retList = new ArrayList<>();
|
||||||
Object[] nodes = aList.toArray();
|
Object[] nodes = aList.toArray();
|
||||||
// Collections.sort would do something similar by calling Arrays.sort
|
// Collections.sort would do something similar by calling Arrays.sort
|
||||||
// internally but would finally iterate through the input list (aList)
|
// internally but would finally iterate through the input list (aList)
|
||||||
@ -420,9 +425,11 @@ private List<NodeId> sortNodes() {
|
|||||||
Arrays.sort(nodes, (Comparator)comparator);
|
Arrays.sort(nodes, (Comparator)comparator);
|
||||||
for (int j=0; j < nodes.length; j++) {
|
for (int j=0; j < nodes.length; j++) {
|
||||||
ClusterNode cNode = (ClusterNode)nodes[j];
|
ClusterNode cNode = (ClusterNode)nodes[j];
|
||||||
// Exclude nodes whose queue is already full.
|
// Only add node to the result list when either condition is met:
|
||||||
if (!cNode.isQueueFull()) {
|
// 1. we don't exclude full nodes
|
||||||
retList.add(cNode.nodeId);
|
// 2. we do exclude full nodes, but the current node is not full
|
||||||
|
if (!excludeFullNodes || !cNode.isQueueFull()) {
|
||||||
|
retList.add(cNode);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return retList;
|
return retList;
|
||||||
|
@ -20,7 +20,6 @@
|
|||||||
|
|
||||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||||
import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
|
import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.NodeQueueLoadMonitor.ClusterNode;
|
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.NodeQueueLoadMonitor.LoadComparator;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.NodeQueueLoadMonitor.LoadComparator;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
Loading…
Reference in New Issue
Block a user