Skip to content

Commit 13c5825

Browse files
YARN-11573. Add config option to make container allocation prefer nodes without reserved containers (#6098)
1 parent 0780710 commit 13c5825

File tree

4 files changed

+152
-1
lines changed

4 files changed

+152
-1
lines changed

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivityDiagnosticConstant.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -107,4 +107,5 @@ public class ActivityDiagnosticConstant {
107107
public final static String
108108
NODE_CAN_NOT_FIND_CONTAINER_TO_BE_UNRESERVED_WHEN_NEEDED =
109109
"Node can't find a container to be unreserved when needed";
110+
public static final String NODE_HAS_BEEN_RESERVED = "Node has been reserved";
110111
}

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -154,6 +154,12 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
154154
@Private
155155
public static final boolean DEFAULT_RESERVE_CONT_LOOK_ALL_NODES = true;
156156

157+
public static final String SKIP_ALLOCATE_ON_NODES_WITH_RESERVED_CONTAINERS = PREFIX
158+
+ "skip-allocate-on-nodes-with-reserved-containers";
159+
160+
@Private
161+
public static final boolean DEFAULT_SKIP_ALLOCATE_ON_NODES_WITH_RESERVED_CONTAINERS = false;
162+
157163
@Private
158164
public static final String MAXIMUM_ALLOCATION = "maximum-allocation";
159165

@@ -938,6 +944,11 @@ public boolean getReservationContinueLook() {
938944
DEFAULT_RESERVE_CONT_LOOK_ALL_NODES);
939945
}
940946

947+
public boolean getSkipAllocateOnNodesWithReservedContainer() {
948+
return getBoolean(SKIP_ALLOCATE_ON_NODES_WITH_RESERVED_CONTAINERS,
949+
DEFAULT_SKIP_ALLOCATE_ON_NODES_WITH_RESERVED_CONTAINERS);
950+
}
951+
941952
private static String getAclKey(QueueACL acl) {
942953
return "acl_" + StringUtils.toLowerCase(acl.toString());
943954
}

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java

Lines changed: 30 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,8 +24,11 @@
2424
import java.util.Optional;
2525

2626
import org.apache.commons.lang3.StringUtils;
27+
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
2728
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityLevel;
2829
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.DiagnosticsCollector;
30+
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
31+
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
2932
import org.slf4j.Logger;
3033
import org.slf4j.LoggerFactory;
3134
import org.apache.hadoop.yarn.api.records.Container;
@@ -850,9 +853,23 @@ private ContainerAllocation allocate(Resource clusterResource,
850853

851854
Iterator<FiCaSchedulerNode> iter = schedulingPS.getPreferredNodeIterator(
852855
candidates);
856+
853857
while (iter.hasNext()) {
854858
FiCaSchedulerNode node = iter.next();
855859

860+
// Do not schedule if there are any reservations to fulfill on the node
861+
if (iter.hasNext() &&
862+
node.getReservedContainer() != null &&
863+
isSkipAllocateOnNodesWithReservedContainer()) {
864+
LOG.debug("Skipping scheduling on node {} since it has already been"
865+
+ " reserved by {}", node.getNodeID(),
866+
node.getReservedContainer().getContainerId());
867+
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
868+
activitiesManager, node, application, schedulerKey,
869+
ActivityDiagnosticConstant.NODE_HAS_BEEN_RESERVED, ActivityLevel.NODE);
870+
continue;
871+
}
872+
856873
if (reservedContainer == null) {
857874
result = preCheckForNodeCandidateSet(node,
858875
schedulingMode, resourceLimits, schedulerKey);
@@ -894,7 +911,19 @@ private ContainerAllocation allocate(Resource clusterResource,
894911

895912
return result;
896913
}
897-
914+
915+
private boolean isSkipAllocateOnNodesWithReservedContainer() {
916+
ResourceScheduler scheduler = rmContext.getScheduler();
917+
boolean skipAllocateOnNodesWithReservedContainer = false;
918+
if (scheduler instanceof CapacityScheduler) {
919+
CapacityScheduler cs = (CapacityScheduler) scheduler;
920+
CapacitySchedulerConfiguration csConf = cs.getConfiguration();
921+
skipAllocateOnNodesWithReservedContainer =
922+
csConf.getSkipAllocateOnNodesWithReservedContainer();
923+
}
924+
return skipAllocateOnNodesWithReservedContainer;
925+
}
926+
898927
@Override
899928
public CSAssignment assignContainers(Resource clusterResource,
900929
CandidateNodeSet<FiCaSchedulerNode> candidates,

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerMultiNodes.java

Lines changed: 110 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,12 +25,16 @@
2525
import java.util.HashSet;
2626
import java.util.Iterator;
2727
import java.util.List;
28+
import java.util.Map;
2829
import java.util.Set;
30+
import java.util.concurrent.ConcurrentMap;
2931
import java.util.concurrent.atomic.AtomicBoolean;
3032

3133
import org.apache.hadoop.thirdparty.com.google.common.collect.Iterators;
3234

35+
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
3336
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
37+
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
3438
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
3539
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
3640
import org.slf4j.Logger;
@@ -478,4 +482,110 @@ public void testMultiNodeSorterAfterHeartbeatInterval() throws Exception {
478482
rm.stop();
479483
}
480484

485+
@Test(timeout=30000)
486+
public void testSkipAllocationOnNodeReservedByAnotherApp() throws Exception {
487+
CapacitySchedulerConfiguration newConf =
488+
new CapacitySchedulerConfiguration(conf);
489+
newConf.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER,
490+
YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER);
491+
newConf.setInt(CapacitySchedulerConfiguration.MULTI_NODE_SORTING_POLICY_NAME
492+
+ ".resource-based.sorting-interval.ms", 0);
493+
newConf.setMaximumApplicationMasterResourcePerQueuePercent("root.default", 1.0f);
494+
newConf.set(CapacitySchedulerConfiguration.SKIP_ALLOCATE_ON_NODES_WITH_RESERVED_CONTAINERS,
495+
"true");
496+
MockRM rm1 = new MockRM(newConf);
497+
498+
rm1.start();
499+
MockNM nm1 = rm1.registerNode("127.0.0.1:1234", 8 * GB);
500+
MockNM nm2 = rm1.registerNode("127.0.0.2:1235", 8 * GB);
501+
502+
// launch an app to queue, AM container should be launched in nm1
503+
RMApp app1 = MockRMAppSubmitter.submit(rm1, MockRMAppSubmissionData.Builder
504+
.createWithMemory(5 * GB, rm1)
505+
.withAppName("app")
506+
.withUser("user")
507+
.withQueue("default")
508+
.build());
509+
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
510+
511+
// launch another app to queue, AM container should be launched in nm2
512+
RMApp app2 = MockRMAppSubmitter.submit(rm1, MockRMAppSubmissionData.Builder
513+
.createWithMemory(5 * GB, rm1)
514+
.withAppName("app")
515+
.withUser("user")
516+
.withQueue("default")
517+
.build());
518+
MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
519+
520+
CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
521+
RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
522+
FiCaSchedulerApp schedulerApp1 =
523+
cs.getApplicationAttempt(am1.getApplicationAttemptId());
524+
FiCaSchedulerApp schedulerApp2 =
525+
cs.getApplicationAttempt(am2.getApplicationAttemptId());
526+
527+
// Ask a container with 4 GB memory size for app1,
528+
am1.allocate("*", 4 * GB, 1, new ArrayList<>());
529+
cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
530+
531+
532+
// Check containers of app1 and app2.
533+
Set<RMNode> reservedContainers = checkReservedContainers(cs,
534+
rm1.getRMContext().getRMNodes(), 1);
535+
Assert.assertEquals(1, reservedContainers.size());
536+
RMNode nodeWithReservedContainer = reservedContainers.iterator().next();
537+
LOG.debug("Reserved container on: {}", nodeWithReservedContainer);
538+
539+
//Move reservation to nm1 for easier testing
540+
if (nodeWithReservedContainer.getNodeID().getHost().startsWith("127.0.0.2")) {
541+
moveReservation(cs, rm1, nm1, nm2, am1);
542+
}
543+
Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
544+
Assert.assertNull(cs.getNode(nm2.getNodeId()).getReservedContainer());
545+
546+
Assert.assertEquals(1, schedulerApp1.getLiveContainers().size());
547+
Assert.assertEquals(1, schedulerApp2.getLiveContainers().size());
548+
Assert.assertEquals(1, schedulerApp1.getReservedContainers().size());
549+
550+
//Make sure to have available headroom on the child queue,
551+
// see: RegularContainerAllocator#checkHeadroom,
552+
//that can make RegularContainerAllocator.preCheckForNodeCandidateSet to return
553+
// ContainerAllocation.QUEUE_SKIPPED
554+
MockNM nm3 = rm1.registerNode("127.0.0.3:1235", 3 * GB);
555+
556+
//Allocate a container for app2, we expect this to be allocated on nm2 as
557+
// nm1 has a reservation for another app
558+
am2.allocate("*", 4 * GB, 1, new ArrayList<>());
559+
cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
560+
Assert.assertNotNull(cs.getNode(nm1.getNodeId()).getReservedContainer());
561+
Assert.assertNotNull(cs.getNode(nm2.getNodeId()).getReservedContainer());
562+
563+
rm1.close();
564+
}
565+
566+
private static void moveReservation(CapacityScheduler cs,
567+
MockRM rm1, MockNM nm1, MockNM nm2, MockAM am1) {
568+
RMNode sourceNode = rm1.getRMContext().getRMNodes().get(nm2.getNodeId());
569+
RMNode targetNode = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
570+
SchedulerApplicationAttempt firstSchedulerAppAttempt =
571+
cs.getApplicationAttempt(am1.getApplicationAttemptId());
572+
FiCaSchedulerApp app = (FiCaSchedulerApp)firstSchedulerAppAttempt;
573+
RMContainer reservedContainer = cs.getNode(sourceNode.getNodeID()).getReservedContainer();
574+
LOG.debug("Moving reservation");
575+
app.moveReservation(reservedContainer,
576+
cs.getNode(sourceNode.getNodeID()), cs.getNode(targetNode.getNodeID()));
577+
}
578+
579+
private static Set<RMNode> checkReservedContainers(CapacityScheduler cs,
580+
ConcurrentMap<NodeId, RMNode> rmNodes, int expectedNumberOfContainers) {
581+
Set<RMNode> result = new HashSet<>();
582+
for (Map.Entry<NodeId, RMNode> entry : rmNodes.entrySet()) {
583+
if (cs.getNode(entry.getKey()).getReservedContainer() != null) {
584+
result.add(entry.getValue());
585+
}
586+
}
587+
588+
Assert.assertEquals(expectedNumberOfContainers, result.size());
589+
return result;
590+
}
481591
}

0 commit comments

Comments
 (0)