YARN-9290. Invalid SchedulingRequest not rejected in Scheduler PlacementConstraintsHandler. Contributed by Prabhu Joseph
This commit is contained in:
parent
828ab400ee
commit
ef950b0863
@ -357,6 +357,9 @@ public void allocate(ApplicationAttemptId appAttemptId,
|
||||
|
||||
response.setContainersFromPreviousAttempts(
|
||||
allocation.getPreviousAttemptContainers());
|
||||
|
||||
response.setRejectedSchedulingRequests(allocation.getRejectedRequest());
|
||||
|
||||
}
|
||||
|
||||
private void handleInvalidResourceException(InvalidResourceRequestException e,
|
||||
|
@ -26,6 +26,7 @@
|
||||
import org.apache.hadoop.yarn.api.records.NMToken;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||
import org.apache.hadoop.yarn.api.records.RejectedSchedulingRequest;
|
||||
|
||||
public class Allocation {
|
||||
|
||||
@ -40,7 +41,7 @@ public class Allocation {
|
||||
final List<Container> demotedContainers;
|
||||
private final List<Container> previousAttemptContainers;
|
||||
private Resource resourceLimit;
|
||||
|
||||
private List<RejectedSchedulingRequest> rejectedRequest;
|
||||
|
||||
public Allocation(List<Container> containers, Resource resourceLimit,
|
||||
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
|
||||
@ -52,17 +53,17 @@ public Allocation(List<Container> containers, Resource resourceLimit,
|
||||
public Allocation(List<Container> containers, Resource resourceLimit,
|
||||
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
|
||||
List<ResourceRequest> fungibleResources, List<NMToken> nmTokens) {
|
||||
this(containers, resourceLimit,strictContainers, fungibleContainers,
|
||||
fungibleResources, nmTokens, null, null, null, null, null);
|
||||
this(containers, resourceLimit, strictContainers, fungibleContainers,
|
||||
fungibleResources, nmTokens, null, null, null, null, null, null);
|
||||
}
|
||||
|
||||
public Allocation(List<Container> containers, Resource resourceLimit,
|
||||
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
|
||||
List<ResourceRequest> fungibleResources, List<NMToken> nmTokens,
|
||||
List<Container> increasedContainers, List<Container> decreasedContainer) {
|
||||
this(containers, resourceLimit,strictContainers, fungibleContainers,
|
||||
this(containers, resourceLimit, strictContainers, fungibleContainers,
|
||||
fungibleResources, nmTokens, increasedContainers, decreasedContainer,
|
||||
null, null, null);
|
||||
null, null, null, null);
|
||||
}
|
||||
|
||||
public Allocation(List<Container> containers, Resource resourceLimit,
|
||||
@ -70,7 +71,8 @@ public Allocation(List<Container> containers, Resource resourceLimit,
|
||||
List<ResourceRequest> fungibleResources, List<NMToken> nmTokens,
|
||||
List<Container> increasedContainers, List<Container> decreasedContainer,
|
||||
List<Container> promotedContainers, List<Container> demotedContainer,
|
||||
List<Container> previousAttemptContainers) {
|
||||
List<Container> previousAttemptContainers, List<RejectedSchedulingRequest>
|
||||
rejectedRequest) {
|
||||
this.containers = containers;
|
||||
this.resourceLimit = resourceLimit;
|
||||
this.strictContainers = strictContainers;
|
||||
@ -82,6 +84,7 @@ public Allocation(List<Container> containers, Resource resourceLimit,
|
||||
this.promotedContainers = promotedContainers;
|
||||
this.demotedContainers = demotedContainer;
|
||||
this.previousAttemptContainers = previousAttemptContainers;
|
||||
this.rejectedRequest = rejectedRequest;
|
||||
}
|
||||
|
||||
public List<Container> getContainers() {
|
||||
@ -128,6 +131,10 @@ public List<Container> getPreviousAttemptContainers() {
|
||||
return previousAttemptContainers;
|
||||
}
|
||||
|
||||
public List<RejectedSchedulingRequest> getRejectedRequest() {
|
||||
return rejectedRequest;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void setResourceLimit(Resource resource) {
|
||||
this.resourceLimit = resource;
|
||||
|
@ -31,6 +31,7 @@
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
@ -42,7 +43,10 @@
|
||||
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||
import org.apache.hadoop.yarn.api.records.RejectionReason;
|
||||
import org.apache.hadoop.yarn.api.records.RejectedSchedulingRequest;
|
||||
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
@ -98,6 +102,7 @@ public class AppSchedulingInfo {
|
||||
public final ContainerUpdateContext updateContext;
|
||||
private final Map<String, String> applicationSchedulingEnvs = new HashMap<>();
|
||||
private final RMContext rmContext;
|
||||
private final int retryAttempts;
|
||||
|
||||
public AppSchedulingInfo(ApplicationAttemptId appAttemptId, String user,
|
||||
Queue queue, AbstractUsersManager abstractUsersManager, long epoch,
|
||||
@ -113,6 +118,9 @@ public AppSchedulingInfo(ApplicationAttemptId appAttemptId, String user,
|
||||
this.appResourceUsage = appResourceUsage;
|
||||
this.applicationSchedulingEnvs.putAll(applicationSchedulingEnvs);
|
||||
this.rmContext = rmContext;
|
||||
this.retryAttempts = rmContext.getYarnConfiguration().getInt(
|
||||
YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_RETRY_ATTEMPTS,
|
||||
YarnConfiguration.DEFAULT_RM_PLACEMENT_CONSTRAINTS_RETRY_ATTEMPTS);
|
||||
|
||||
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
||||
updateContext = new ContainerUpdateContext(this);
|
||||
@ -496,6 +504,20 @@ public List<SchedulingRequest> getAllSchedulingRequests() {
|
||||
return ret;
|
||||
}
|
||||
|
||||
public List<RejectedSchedulingRequest> getRejectedRequest() {
|
||||
this.readLock.lock();
|
||||
try {
|
||||
return schedulerKeyToAppPlacementAllocator.values().stream()
|
||||
.filter(ap -> ap.getPlacementAttempt() >= retryAttempts)
|
||||
.map(ap -> RejectedSchedulingRequest.newInstance(
|
||||
RejectionReason.COULD_NOT_SCHEDULE_ON_NODE,
|
||||
ap.getSchedulingRequest()))
|
||||
.collect(Collectors.toList());
|
||||
} finally {
|
||||
this.readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
public PendingAsk getNextPendingAsk() {
|
||||
readLock.lock();
|
||||
try {
|
||||
@ -780,8 +802,8 @@ public boolean precheckNode(SchedulerRequestKey schedulerKey,
|
||||
try {
|
||||
AppPlacementAllocator ap =
|
||||
schedulerKeyToAppPlacementAllocator.get(schedulerKey);
|
||||
return (ap != null) && ap.precheckNode(schedulerNode,
|
||||
schedulingMode, dcOpt);
|
||||
return (ap != null) && (ap.getPlacementAttempt() < retryAttempts) &&
|
||||
ap.precheckNode(schedulerNode, schedulingMode, dcOpt);
|
||||
} finally {
|
||||
this.readLock.unlock();
|
||||
}
|
||||
|
@ -821,7 +821,7 @@ public Allocation getAllocation(ResourceCalculator resourceCalculator,
|
||||
currentContPreemption, Collections.singletonList(rr), updatedNMTokens,
|
||||
newlyIncreasedContainers, newlyDecreasedContainers,
|
||||
newlyPromotedContainers, newlyDemotedContainers,
|
||||
previousAttemptContainers);
|
||||
previousAttemptContainers, appSchedulingInfo.getRejectedRequest());
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
|
@ -951,7 +951,7 @@ public Allocation allocate(ApplicationAttemptId appAttemptId,
|
||||
updatedNMTokens, null, null,
|
||||
application.pullNewlyPromotedContainers(),
|
||||
application.pullNewlyDemotedContainers(),
|
||||
previousAttemptContainers);
|
||||
previousAttemptContainers, null);
|
||||
}
|
||||
|
||||
private List<MaxResourceValidationResult> validateResourceRequests(
|
||||
|
@ -34,6 +34,7 @@
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
@ -57,6 +58,7 @@ public abstract class AppPlacementAllocator<N extends SchedulerNode> {
|
||||
protected AppSchedulingInfo appSchedulingInfo;
|
||||
protected SchedulerRequestKey schedulerRequestKey;
|
||||
protected RMContext rmContext;
|
||||
private AtomicInteger placementAttempt = new AtomicInteger(0);
|
||||
|
||||
/**
|
||||
* Get iterator of preferred node depends on requirement and/or availability.
|
||||
@ -205,4 +207,12 @@ public void initialize(AppSchedulingInfo appSchedulingInfo,
|
||||
* @return SchedulingRequest
|
||||
*/
|
||||
public abstract SchedulingRequest getSchedulingRequest();
|
||||
|
||||
public int getPlacementAttempt() {
|
||||
return placementAttempt.get();
|
||||
}
|
||||
|
||||
public void incrementPlacementAttempt() {
|
||||
placementAttempt.getAndIncrement();
|
||||
}
|
||||
}
|
||||
|
@ -363,6 +363,7 @@ private boolean checkCardinalityAndPending(SchedulerNode node,
|
||||
placementConstraintManager, allocationTagsManager, dcOpt);
|
||||
} catch (InvalidAllocationTagsQueryException e) {
|
||||
LOG.warn("Failed to query node cardinality:", e);
|
||||
this.incrementPlacementAttempt();
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
@ -28,6 +28,7 @@
|
||||
import org.apache.hadoop.yarn.api.records.Priority;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSLeafQueue;
|
||||
@ -44,10 +45,12 @@ public void testBacklistChanged() {
|
||||
ApplicationAttemptId.newInstance(appIdImpl, 1);
|
||||
|
||||
FSLeafQueue queue = mock(FSLeafQueue.class);
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
doReturn("test").when(queue).getQueueName();
|
||||
doReturn(new YarnConfiguration()).when(rmContext).getYarnConfiguration();
|
||||
AppSchedulingInfo appSchedulingInfo = new AppSchedulingInfo(appAttemptId,
|
||||
"test", queue, null, 0, new ResourceUsage(),
|
||||
new HashMap<String, String>(), null);
|
||||
new HashMap<String, String>(), rmContext);
|
||||
|
||||
appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
|
||||
new ArrayList<String>());
|
||||
@ -117,9 +120,11 @@ public void testSchedulerKeyAccounting() {
|
||||
|
||||
Queue queue = mock(Queue.class);
|
||||
doReturn(mock(QueueMetrics.class)).when(queue).getMetrics();
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
doReturn(new YarnConfiguration()).when(rmContext).getYarnConfiguration();
|
||||
AppSchedulingInfo info = new AppSchedulingInfo(
|
||||
appAttemptId, "test", queue, mock(ActiveUsersManager.class), 0,
|
||||
new ResourceUsage(), new HashMap<>(), mock(RMContext.class));
|
||||
new ResourceUsage(), new HashMap<>(), rmContext);
|
||||
Assert.assertEquals(0, info.getSchedulerKeys().size());
|
||||
|
||||
Priority pri1 = Priority.newInstance(1);
|
||||
|
@ -75,6 +75,7 @@ public void testActiveUsersWhenMove() {
|
||||
ApplicationAttemptId appAttId = createAppAttemptId(0, 0);
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
when(rmContext.getEpoch()).thenReturn(3L);
|
||||
when(rmContext.getYarnConfiguration()).thenReturn(conf);
|
||||
SchedulerApplicationAttempt app = new SchedulerApplicationAttempt(appAttId,
|
||||
user, queue1, queue1.getAbstractUsersManager(), rmContext);
|
||||
|
||||
@ -121,6 +122,7 @@ public void testMove() {
|
||||
ApplicationAttemptId appAttId = createAppAttemptId(0, 0);
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
when(rmContext.getEpoch()).thenReturn(3L);
|
||||
when(rmContext.getYarnConfiguration()).thenReturn(conf);
|
||||
SchedulerApplicationAttempt app = new SchedulerApplicationAttempt(appAttId,
|
||||
user, oldQueue, oldQueue.getAbstractUsersManager(), rmContext);
|
||||
oldMetrics.submitApp(user);
|
||||
@ -242,6 +244,7 @@ public void testAppPercentages() throws Exception {
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
when(rmContext.getEpoch()).thenReturn(3L);
|
||||
when(rmContext.getScheduler()).thenReturn(scheduler);
|
||||
when(rmContext.getYarnConfiguration()).thenReturn(conf);
|
||||
|
||||
final String user = "user1";
|
||||
Queue queue = createQueue("test", null);
|
||||
@ -300,6 +303,7 @@ public void testAppPercentagesOnswitch() throws Exception {
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
when(rmContext.getEpoch()).thenReturn(3L);
|
||||
when(rmContext.getScheduler()).thenReturn(scheduler);
|
||||
when(rmContext.getYarnConfiguration()).thenReturn(conf);
|
||||
|
||||
final String user = "user1";
|
||||
Queue queue = createQueue("test", null);
|
||||
@ -322,6 +326,7 @@ public void testSchedulingOpportunityOverflow() throws Exception {
|
||||
Queue queue = createQueue("test", null);
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
when(rmContext.getEpoch()).thenReturn(3L);
|
||||
when(rmContext.getYarnConfiguration()).thenReturn(conf);
|
||||
SchedulerApplicationAttempt app = new SchedulerApplicationAttempt(
|
||||
attemptId, "user", queue, queue.getAbstractUsersManager(), rmContext);
|
||||
Priority priority = Priority.newInstance(1);
|
||||
@ -347,6 +352,7 @@ public void testHasPendingResourceRequest() throws Exception {
|
||||
Queue queue = createQueue("test", null);
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
when(rmContext.getEpoch()).thenReturn(3L);
|
||||
when(rmContext.getYarnConfiguration()).thenReturn(conf);
|
||||
SchedulerApplicationAttempt app = new SchedulerApplicationAttempt(
|
||||
attemptId, "user", queue, queue.getAbstractUsersManager(), rmContext);
|
||||
|
||||
|
@ -4130,6 +4130,7 @@ public void testApplicationQueuePercent()
|
||||
RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
|
||||
when(nlm.getResourceByLabel(any(), any())).thenReturn(res);
|
||||
when(rmContext.getNodeLabelManager()).thenReturn(nlm);
|
||||
when(rmContext.getYarnConfiguration()).thenReturn(csConf);
|
||||
|
||||
// Queue "test" consumes 100% of the cluster, so its capacity and absolute
|
||||
// capacity are both 1.0f.
|
||||
|
@ -448,6 +448,59 @@ public RMNodeLabelsManager createNodeLabelManager() {
|
||||
rm1.close();
|
||||
}
|
||||
|
||||
@Test(timeout = 30000L)
|
||||
public void testInvalidSchedulingRequest() throws Exception {
|
||||
|
||||
Configuration csConf = TestUtils.getConfigurationWithMultipleQueues(conf);
|
||||
MockRM rm1 = new MockRM(csConf) {
|
||||
@Override
|
||||
public RMNodeLabelsManager createNodeLabelManager() {
|
||||
return mgr;
|
||||
}
|
||||
};
|
||||
|
||||
rm1.getRMContext().setNodeLabelManager(mgr);
|
||||
rm1.start();
|
||||
|
||||
// 4 NMs.
|
||||
MockNM[] nms = new MockNM[4];
|
||||
RMNode[] rmNodes = new RMNode[4];
|
||||
for (int i = 0; i < 4; i++) {
|
||||
nms[i] = rm1.registerNode("192.168.0." + i + ":1234", 10 * GB);
|
||||
rmNodes[i] = rm1.getRMContext().getRMNodes().get(nms[i].getNodeId());
|
||||
}
|
||||
|
||||
RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "c");
|
||||
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nms[0]);
|
||||
|
||||
// Constraint with Invalid Allocation Tag Namespace
|
||||
PlacementConstraint constraint = targetNotIn("node",
|
||||
allocationTagWithNamespace("invalid", "t1")).build();
|
||||
SchedulingRequest sc = SchedulingRequest
|
||||
.newInstance(1, Priority.newInstance(1),
|
||||
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED),
|
||||
ImmutableSet.of("t1"),
|
||||
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)),
|
||||
constraint);
|
||||
AllocateRequest request = AllocateRequest.newBuilder()
|
||||
.schedulingRequests(ImmutableList.of(sc)).build();
|
||||
am1.allocate(request);
|
||||
|
||||
try {
|
||||
GenericTestUtils.waitFor(() -> {
|
||||
try {
|
||||
doNodeHeartbeat(nms);
|
||||
AllocateResponse response = am1.schedule();
|
||||
return response.getRejectedSchedulingRequests().size() == 1;
|
||||
} catch (Exception e) {
|
||||
return false;
|
||||
}
|
||||
}, 500, 20000);
|
||||
} catch (Exception e) {
|
||||
Assert.fail("Failed to reject invalid scheduling request");
|
||||
}
|
||||
}
|
||||
|
||||
private static void doNodeHeartbeat(MockNM... nms) throws Exception {
|
||||
for (MockNM nm : nms) {
|
||||
nm.nodeHeartbeat(true);
|
||||
|
@ -121,6 +121,7 @@ public Resource answer(InvocationOnMock invocation) throws Throwable {
|
||||
}
|
||||
});
|
||||
|
||||
rmContext.setYarnConfiguration(conf);
|
||||
rmContext.setNodeLabelManager(nlm);
|
||||
rmContext.setSystemMetricsPublisher(mock(SystemMetricsPublisher.class));
|
||||
rmContext.setRMApplicationHistoryWriter(mock(RMApplicationHistoryWriter.class));
|
||||
|
@ -375,6 +375,7 @@ public void testNoNextPendingAsk() {
|
||||
Mockito.when(rmApp.getApplicationSubmissionContext())
|
||||
.thenReturn(appContext);
|
||||
Mockito.when(rmContext.getRMApps()).thenReturn(rmApps);
|
||||
Mockito.when(rmContext.getYarnConfiguration()).thenReturn(conf);
|
||||
FSAppAttempt schedulerApp =
|
||||
new FSAppAttempt(scheduler, applicationAttemptId, "user1", queue,
|
||||
null, rmContext);
|
||||
|
@ -52,6 +52,7 @@ public void setup() {
|
||||
rmContext = mock(RMContext.class);
|
||||
when(rmContext.getQueuePlacementManager()).thenReturn(placementManager);
|
||||
when(rmContext.getEpoch()).thenReturn(0L);
|
||||
when(rmContext.getYarnConfiguration()).thenReturn(conf);
|
||||
clock = new ControlledClock();
|
||||
scheduler = mock(FairScheduler.class);
|
||||
when(scheduler.getConf()).thenReturn(conf);
|
||||
|
@ -571,6 +571,7 @@ public void testApplicationAssignmentPreventsRemovalOfDynamicQueue() {
|
||||
ActiveUsersManager activeUsersManager =
|
||||
Mockito.mock(ActiveUsersManager.class);
|
||||
RMContext rmContext = Mockito.mock(RMContext.class);
|
||||
doReturn(scheduler.getConfig()).when(rmContext).getYarnConfiguration();
|
||||
|
||||
// the appAttempt is created
|
||||
// removeEmptyDynamicQueues() should not remove the queue
|
||||
|
@ -193,6 +193,7 @@ public void testAppAttemptMetrics() throws Exception {
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
((RMContextImpl) rmContext).setScheduler(scheduler);
|
||||
((RMContextImpl) rmContext).setYarnConfiguration(conf);
|
||||
scheduler.setRMContext(rmContext);
|
||||
scheduler.init(conf);
|
||||
scheduler.start();
|
||||
|
Loading…
Reference in New Issue
Block a user