|
@@ -18,22 +18,20 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity;
|
|
|
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.Arrays;
|
|
|
+import java.util.Collection;
|
|
|
+import java.util.LinkedHashMap;
|
|
|
+import java.util.Map;
|
|
|
+
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
|
|
|
-
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
|
|
|
- .ParentQueue;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
|
|
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.ResourceUtils;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
|
|
|
-import java.util.ArrayList;
|
|
|
-import java.util.Arrays;
|
|
|
-import java.util.Collection;
|
|
|
-import java.util.LinkedHashMap;
|
|
|
-import java.util.Map;
|
|
|
-
|
|
|
/**
|
|
|
* Temporary data-structure tracking resource availability, pending resource
|
|
|
* need, current utilization. This is per-queue-per-partition data structure
|
|
@@ -74,7 +72,8 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
|
|
|
// idealAssigned, used etc.
|
|
|
Map<String, TempUserPerPartition> usersPerPartition = new LinkedHashMap<>();
|
|
|
|
|
|
- TempQueuePerPartition(String queueName, Resource current,
|
|
|
+ @SuppressWarnings("checkstyle:parameternumber")
|
|
|
+ public TempQueuePerPartition(String queueName, Resource current,
|
|
|
boolean preemptionDisabled, String partition, Resource killable,
|
|
|
float absCapacity, float absMaxCapacity, Resource totalPartitionResource,
|
|
|
Resource reserved, CSQueue queue, Resource effMinRes,
|
|
@@ -94,7 +93,7 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
|
|
|
pendingDeductReserved = Resources.createResource(0);
|
|
|
}
|
|
|
|
|
|
- if (ParentQueue.class.isAssignableFrom(queue.getClass())) {
|
|
|
+ if (queue != null && ParentQueue.class.isAssignableFrom(queue.getClass())) {
|
|
|
parentQueue = (ParentQueue) queue;
|
|
|
}
|
|
|
|
|
@@ -179,15 +178,14 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
|
|
|
// Because for a satisfied parent queue, it could have some under-utilized
|
|
|
// leaf queues. Such under-utilized leaf queue could preemption resources
|
|
|
// from over-utilized leaf queue located at other hierarchies.
|
|
|
- if (null == children || children.isEmpty()) {
|
|
|
- Resource maxOfGuranteedAndUsedDeductAssigned = Resources.subtract(
|
|
|
- Resources.max(rc, clusterResource, getUsed(), getGuaranteed()),
|
|
|
- idealAssigned);
|
|
|
- maxOfGuranteedAndUsedDeductAssigned = Resources.max(rc, clusterResource,
|
|
|
- maxOfGuranteedAndUsedDeductAssigned, Resources.none());
|
|
|
- accepted = Resources.min(rc, clusterResource, accepted,
|
|
|
- maxOfGuranteedAndUsedDeductAssigned);
|
|
|
- }
|
|
|
+
|
|
|
+ accepted = filterByMaxDeductAssigned(rc, clusterResource, accepted);
|
|
|
+
|
|
|
+ // accepted so far contains the "quota acceptable" amount, we now filter by
|
|
|
+ // locality acceptable
|
|
|
+
|
|
|
+ accepted = acceptedByLocality(rc, accepted);
|
|
|
+
|
|
|
Resource remain = Resources.subtract(avail, accepted);
|
|
|
Resources.addTo(idealAssigned, accepted);
|
|
|
return remain;
|
|
@@ -329,4 +327,72 @@ public class TempQueuePerPartition extends AbstractPreemptionEntity {
|
|
|
public Map<String, TempUserPerPartition> getUsersPerPartition() {
|
|
|
return usersPerPartition;
|
|
|
}
|
|
|
+
|
|
|
+ public void setPending(Resource pending) {
|
|
|
+ this.pending = pending;
|
|
|
+ }
|
|
|
+
|
|
|
+ public Resource getIdealAssigned() {
|
|
|
+ return idealAssigned;
|
|
|
+ }
|
|
|
+
|
|
|
+ public String toGlobalString() {
|
|
|
+ StringBuilder sb = new StringBuilder();
|
|
|
+ sb.append("\n").append(toString());
|
|
|
+ for (TempQueuePerPartition c : children) {
|
|
|
+ sb.append(c.toGlobalString());
|
|
|
+ }
|
|
|
+ return sb.toString();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This method is visible to allow sub-classes to override the behavior,
|
|
|
+ * specifically to take into account locality-based limitations of how much
|
|
|
+ * the queue can consumed.
|
|
|
+ *
|
|
|
+ * @param rc the ResourceCalculator to be used.
|
|
|
+ * @param offered the input amount of Resource offered to this queue.
|
|
|
+ *
|
|
|
+ * @return the subset of Resource(s) that the queue can consumed after
|
|
|
+ * accounting for locality effects.
|
|
|
+ */
|
|
|
+ protected Resource acceptedByLocality(ResourceCalculator rc,
|
|
|
+ Resource offered) {
|
|
|
+ return offered;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This method is visible to allow sub-classes to override the behavior,
|
|
|
+ * specifically for federation purposes we do not want to cap resources as it
|
|
|
+ * is done here.
|
|
|
+ *
|
|
|
+ * @param rc the {@code ResourceCalculator} to be used
|
|
|
+ * @param clusterResource the total cluster resources
|
|
|
+ * @param offered the resources offered to this queue
|
|
|
+ * @return the amount of resources accepted after considering max and
|
|
|
+ * deducting assigned.
|
|
|
+ */
|
|
|
+ protected Resource filterByMaxDeductAssigned(ResourceCalculator rc,
|
|
|
+ Resource clusterResource, Resource offered) {
|
|
|
+ if (null == children || children.isEmpty()) {
|
|
|
+ Resource maxOfGuranteedAndUsedDeductAssigned = Resources.subtract(
|
|
|
+ Resources.max(rc, clusterResource, getUsed(), getGuaranteed()),
|
|
|
+ idealAssigned);
|
|
|
+ maxOfGuranteedAndUsedDeductAssigned = Resources.max(rc, clusterResource,
|
|
|
+ maxOfGuranteedAndUsedDeductAssigned, Resources.none());
|
|
|
+ offered = Resources.min(rc, clusterResource, offered,
|
|
|
+ maxOfGuranteedAndUsedDeductAssigned);
|
|
|
+ }
|
|
|
+ return offered;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This method is visible to allow sub-classes to ovverride the behavior,
|
|
|
+ * specifically for federation purposes we need to initialize per-sub-cluster
|
|
|
+ * roots as well as the global one.
|
|
|
+ */
|
|
|
+ protected void initializeRootIdealWithGuarangeed() {
|
|
|
+ idealAssigned = Resources.clone(getGuaranteed());
|
|
|
+ }
|
|
|
+
|
|
|
}
|