|
@@ -31,36 +31,47 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
import java.util.Comparator;
|
|
import java.util.Comparator;
|
|
-import java.util.HashMap;
|
|
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.Map;
|
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
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;
|
|
|
|
|
|
-public class TopKNodeSelector implements ClusterMonitor {
|
|
|
|
|
|
+/**
|
|
|
|
+ * The NodeQueueLoadMonitor keeps track of load metrics (such as queue length
|
|
|
|
+ * and total wait time) associated with Container Queues on the Node Manager.
|
|
|
|
+ * It uses this information to periodically sort the Nodes from least to most
|
|
|
|
+ * loaded.
|
|
|
|
+ */
|
|
|
|
+public class NodeQueueLoadMonitor implements ClusterMonitor {
|
|
|
|
|
|
- final static Log LOG = LogFactory.getLog(TopKNodeSelector.class);
|
|
|
|
|
|
+ final static Log LOG = LogFactory.getLog(NodeQueueLoadMonitor.class);
|
|
|
|
|
|
- public enum TopKComparator implements Comparator<ClusterNode> {
|
|
|
|
- WAIT_TIME,
|
|
|
|
- QUEUE_LENGTH;
|
|
|
|
|
|
+ /**
|
|
|
|
+ * The comparator used to specify the metric against which the load
|
|
|
|
+ * of two Nodes are compared.
|
|
|
|
+ */
|
|
|
|
+ public enum LoadComparator implements Comparator<ClusterNode> {
|
|
|
|
+ QUEUE_LENGTH,
|
|
|
|
+ QUEUE_WAIT_TIME;
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public int compare(ClusterNode o1, ClusterNode o2) {
|
|
public int compare(ClusterNode o1, ClusterNode o2) {
|
|
- if (getQuant(o1) == getQuant(o2)) {
|
|
|
|
|
|
+ if (getMetric(o1) == getMetric(o2)) {
|
|
return o1.timestamp < o2.timestamp ? +1 : -1;
|
|
return o1.timestamp < o2.timestamp ? +1 : -1;
|
|
}
|
|
}
|
|
- return getQuant(o1) > getQuant(o2) ? +1 : -1;
|
|
|
|
|
|
+ return getMetric(o1) > getMetric(o2) ? +1 : -1;
|
|
}
|
|
}
|
|
|
|
|
|
- private int getQuant(ClusterNode c) {
|
|
|
|
- return (this == WAIT_TIME) ? c.queueTime : c.waitQueueLength;
|
|
|
|
|
|
+ public int getMetric(ClusterNode c) {
|
|
|
|
+ return (this == QUEUE_LENGTH) ? c.queueLength : c.queueWaitTime;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
static class ClusterNode {
|
|
static class ClusterNode {
|
|
- int queueTime = -1;
|
|
|
|
- int waitQueueLength = 0;
|
|
|
|
|
|
+ int queueLength = 0;
|
|
|
|
+ int queueWaitTime = -1;
|
|
double timestamp;
|
|
double timestamp;
|
|
final NodeId nodeId;
|
|
final NodeId nodeId;
|
|
|
|
|
|
@@ -69,13 +80,13 @@ public class TopKNodeSelector implements ClusterMonitor {
|
|
updateTimestamp();
|
|
updateTimestamp();
|
|
}
|
|
}
|
|
|
|
|
|
- public ClusterNode setQueueTime(int queueTime) {
|
|
|
|
- this.queueTime = queueTime;
|
|
|
|
|
|
+ public ClusterNode setQueueLength(int qLength) {
|
|
|
|
+ this.queueLength = qLength;
|
|
return this;
|
|
return this;
|
|
}
|
|
}
|
|
|
|
|
|
- public ClusterNode setWaitQueueLength(int queueLength) {
|
|
|
|
- this.waitQueueLength = queueLength;
|
|
|
|
|
|
+ public ClusterNode setQueueWaitTime(int wTime) {
|
|
|
|
+ this.queueWaitTime = wTime;
|
|
return this;
|
|
return this;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -85,34 +96,37 @@ public class TopKNodeSelector implements ClusterMonitor {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private final int k;
|
|
|
|
- private final List<NodeId> topKNodes;
|
|
|
|
private final ScheduledExecutorService scheduledExecutor;
|
|
private final ScheduledExecutorService scheduledExecutor;
|
|
- private final HashMap<NodeId, ClusterNode> clusterNodes = new HashMap<>();
|
|
|
|
- private final Comparator<ClusterNode> comparator;
|
|
|
|
|
|
+
|
|
|
|
+ private final List<NodeId> sortedNodes;
|
|
|
|
+ private final Map<NodeId, ClusterNode> clusterNodes =
|
|
|
|
+ new ConcurrentHashMap<>();
|
|
|
|
+ private final LoadComparator comparator;
|
|
|
|
+ private QueueLimitCalculator thresholdCalculator;
|
|
|
|
|
|
Runnable computeTask = new Runnable() {
|
|
Runnable computeTask = new Runnable() {
|
|
@Override
|
|
@Override
|
|
public void run() {
|
|
public void run() {
|
|
- synchronized (topKNodes) {
|
|
|
|
- topKNodes.clear();
|
|
|
|
- topKNodes.addAll(computeTopKNodes());
|
|
|
|
|
|
+ synchronized (sortedNodes) {
|
|
|
|
+ sortedNodes.clear();
|
|
|
|
+ sortedNodes.addAll(sortNodes());
|
|
|
|
+ if (thresholdCalculator != null) {
|
|
|
|
+ thresholdCalculator.update();
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
};
|
|
};
|
|
|
|
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
- TopKNodeSelector(int k, TopKComparator comparator) {
|
|
|
|
- this.k = k;
|
|
|
|
- this.topKNodes = new ArrayList<>();
|
|
|
|
|
|
+ NodeQueueLoadMonitor(LoadComparator comparator) {
|
|
|
|
+ this.sortedNodes = new ArrayList<>();
|
|
this.comparator = comparator;
|
|
this.comparator = comparator;
|
|
this.scheduledExecutor = null;
|
|
this.scheduledExecutor = null;
|
|
}
|
|
}
|
|
|
|
|
|
- public TopKNodeSelector(int k, long nodeComputationInterval,
|
|
|
|
- TopKComparator comparator) {
|
|
|
|
- this.k = k;
|
|
|
|
- this.topKNodes = new ArrayList<>();
|
|
|
|
|
|
+ public NodeQueueLoadMonitor(long nodeComputationInterval,
|
|
|
|
+ LoadComparator comparator) {
|
|
|
|
+ this.sortedNodes = new ArrayList<>();
|
|
this.scheduledExecutor = Executors.newScheduledThreadPool(1);
|
|
this.scheduledExecutor = Executors.newScheduledThreadPool(1);
|
|
this.comparator = comparator;
|
|
this.comparator = comparator;
|
|
this.scheduledExecutor.scheduleAtFixedRate(computeTask,
|
|
this.scheduledExecutor.scheduleAtFixedRate(computeTask,
|
|
@@ -120,12 +134,32 @@ public class TopKNodeSelector implements ClusterMonitor {
|
|
TimeUnit.MILLISECONDS);
|
|
TimeUnit.MILLISECONDS);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ List<NodeId> getSortedNodes() {
|
|
|
|
+ return sortedNodes;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public QueueLimitCalculator getThresholdCalculator() {
|
|
|
|
+ return thresholdCalculator;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ Map<NodeId, ClusterNode> getClusterNodes() {
|
|
|
|
+ return clusterNodes;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ Comparator<ClusterNode> getComparator() {
|
|
|
|
+ return comparator;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void initThresholdCalculator(float sigma, int limitMin, int limitMax) {
|
|
|
|
+ this.thresholdCalculator =
|
|
|
|
+ new QueueLimitCalculator(this, sigma, limitMin, limitMax);
|
|
|
|
+ }
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void addNode(List<NMContainerStatus> containerStatuses, RMNode
|
|
public void addNode(List<NMContainerStatus> containerStatuses, RMNode
|
|
rmNode) {
|
|
rmNode) {
|
|
LOG.debug("Node added event from: " + rmNode.getNode().getName());
|
|
LOG.debug("Node added event from: " + rmNode.getNode().getName());
|
|
- // Ignoring this currently : atleast one NODE_UPDATE heartbeat is
|
|
|
|
|
|
+ // Ignoring this currently : at least one NODE_UPDATE heartbeat is
|
|
// required to ensure node eligibility.
|
|
// required to ensure node eligibility.
|
|
}
|
|
}
|
|
|
|
|
|
@@ -143,24 +177,24 @@ public class TopKNodeSelector implements ClusterMonitor {
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public void nodeUpdate(RMNode rmNode) {
|
|
|
|
|
|
+ public void updateNode(RMNode rmNode) {
|
|
LOG.debug("Node update event from: " + rmNode.getNodeID());
|
|
LOG.debug("Node update event from: " + rmNode.getNodeID());
|
|
QueuedContainersStatus queuedContainersStatus =
|
|
QueuedContainersStatus queuedContainersStatus =
|
|
rmNode.getQueuedContainersStatus();
|
|
rmNode.getQueuedContainersStatus();
|
|
int estimatedQueueWaitTime =
|
|
int estimatedQueueWaitTime =
|
|
queuedContainersStatus.getEstimatedQueueWaitTime();
|
|
queuedContainersStatus.getEstimatedQueueWaitTime();
|
|
int waitQueueLength = queuedContainersStatus.getWaitQueueLength();
|
|
int waitQueueLength = queuedContainersStatus.getWaitQueueLength();
|
|
- // Add nodes to clusterNodes.. if estimatedQueueTime is -1, Ignore node
|
|
|
|
- // UNLESS comparator is based on queue length, in which case, we should add
|
|
|
|
|
|
+ // Add nodes to clusterNodes. If estimatedQueueTime is -1, ignore node
|
|
|
|
+ // UNLESS comparator is based on queue length.
|
|
synchronized (this.clusterNodes) {
|
|
synchronized (this.clusterNodes) {
|
|
ClusterNode currentNode = this.clusterNodes.get(rmNode.getNodeID());
|
|
ClusterNode currentNode = this.clusterNodes.get(rmNode.getNodeID());
|
|
if (currentNode == null) {
|
|
if (currentNode == null) {
|
|
if (estimatedQueueWaitTime != -1
|
|
if (estimatedQueueWaitTime != -1
|
|
- || comparator == TopKComparator.QUEUE_LENGTH) {
|
|
|
|
|
|
+ || comparator == LoadComparator.QUEUE_LENGTH) {
|
|
this.clusterNodes.put(rmNode.getNodeID(),
|
|
this.clusterNodes.put(rmNode.getNodeID(),
|
|
new ClusterNode(rmNode.getNodeID())
|
|
new ClusterNode(rmNode.getNodeID())
|
|
- .setQueueTime(estimatedQueueWaitTime)
|
|
|
|
- .setWaitQueueLength(waitQueueLength));
|
|
|
|
|
|
+ .setQueueWaitTime(estimatedQueueWaitTime)
|
|
|
|
+ .setQueueLength(waitQueueLength));
|
|
LOG.info("Inserting ClusterNode [" + rmNode.getNodeID() + "]" +
|
|
LOG.info("Inserting ClusterNode [" + rmNode.getNodeID() + "]" +
|
|
"with queue wait time [" + estimatedQueueWaitTime + "] and " +
|
|
"with queue wait time [" + estimatedQueueWaitTime + "] and " +
|
|
"wait queue length [" + waitQueueLength + "]");
|
|
"wait queue length [" + waitQueueLength + "]");
|
|
@@ -171,10 +205,10 @@ public class TopKNodeSelector implements ClusterMonitor {
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
if (estimatedQueueWaitTime != -1
|
|
if (estimatedQueueWaitTime != -1
|
|
- || comparator == TopKComparator.QUEUE_LENGTH) {
|
|
|
|
|
|
+ || comparator == LoadComparator.QUEUE_LENGTH) {
|
|
currentNode
|
|
currentNode
|
|
- .setQueueTime(estimatedQueueWaitTime)
|
|
|
|
- .setWaitQueueLength(waitQueueLength)
|
|
|
|
|
|
+ .setQueueWaitTime(estimatedQueueWaitTime)
|
|
|
|
+ .setQueueLength(waitQueueLength)
|
|
.updateTimestamp();
|
|
.updateTimestamp();
|
|
LOG.info("Updating ClusterNode [" + rmNode.getNodeID() + "]" +
|
|
LOG.info("Updating ClusterNode [" + rmNode.getNodeID() + "]" +
|
|
"with queue wait time [" + estimatedQueueWaitTime + "] and " +
|
|
"with queue wait time [" + estimatedQueueWaitTime + "] and " +
|
|
@@ -182,8 +216,8 @@ public class TopKNodeSelector implements ClusterMonitor {
|
|
} else {
|
|
} else {
|
|
this.clusterNodes.remove(rmNode.getNodeID());
|
|
this.clusterNodes.remove(rmNode.getNodeID());
|
|
LOG.info("Deleting ClusterNode [" + rmNode.getNodeID() + "]" +
|
|
LOG.info("Deleting ClusterNode [" + rmNode.getNodeID() + "]" +
|
|
- "with queue wait time [" + currentNode.queueTime + "] and " +
|
|
|
|
- "wait queue length [" + currentNode.waitQueueLength + "]");
|
|
|
|
|
|
+ "with queue wait time [" + currentNode.queueWaitTime + "] and " +
|
|
|
|
+ "wait queue length [" + currentNode.queueLength + "]");
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -192,25 +226,38 @@ public class TopKNodeSelector implements ClusterMonitor {
|
|
@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());
|
|
- // Ignoring this currently...
|
|
|
|
|
|
+ // Ignoring this currently.
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Returns all Node Ids as ordered list from Least to Most Loaded.
|
|
|
|
+ * @return ordered list of nodes
|
|
|
|
+ */
|
|
public List<NodeId> selectNodes() {
|
|
public List<NodeId> selectNodes() {
|
|
- synchronized (this.topKNodes) {
|
|
|
|
- return this.k < this.topKNodes.size() ?
|
|
|
|
- new ArrayList<>(this.topKNodes).subList(0, this.k) :
|
|
|
|
- new ArrayList<>(this.topKNodes);
|
|
|
|
|
|
+ return selectLeastLoadedNodes(-1);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Returns 'K' of the least Loaded Node Ids as ordered list.
|
|
|
|
+ * @param k max number of nodes to return
|
|
|
|
+ * @return ordered list of nodes
|
|
|
|
+ */
|
|
|
|
+ public List<NodeId> selectLeastLoadedNodes(int k) {
|
|
|
|
+ synchronized (this.sortedNodes) {
|
|
|
|
+ return ((k < this.sortedNodes.size()) && (k >= 0)) ?
|
|
|
|
+ new ArrayList<>(this.sortedNodes).subList(0, k) :
|
|
|
|
+ new ArrayList<>(this.sortedNodes);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private List<NodeId> computeTopKNodes() {
|
|
|
|
|
|
+ private List<NodeId> sortNodes() {
|
|
synchronized (this.clusterNodes) {
|
|
synchronized (this.clusterNodes) {
|
|
ArrayList aList = new ArrayList<>(this.clusterNodes.values());
|
|
ArrayList aList = new ArrayList<>(this.clusterNodes.values());
|
|
List<NodeId> retList = new ArrayList<>();
|
|
List<NodeId> 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)
|
|
- // to reset the value of each element.. Since we don't really care about
|
|
|
|
|
|
+ // to reset the value of each element. Since we don't really care about
|
|
// 'aList', we can use the iteration to create the list of nodeIds which
|
|
// 'aList', we can use the iteration to create the list of nodeIds which
|
|
// is what we ultimately care about.
|
|
// is what we ultimately care about.
|
|
Arrays.sort(nodes, (Comparator)comparator);
|
|
Arrays.sort(nodes, (Comparator)comparator);
|
|
@@ -220,4 +267,5 @@ public class TopKNodeSelector implements ClusterMonitor {
|
|
return retList;
|
|
return retList;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
}
|
|
}
|