Skip to content

Commit

Permalink
YARN-9290. Invalid SchedulingRequest not rejected in Scheduler Placem…
Browse files Browse the repository at this point in the history
…entConstraintsHandler. Contributed by Prabhu Joseph
  • Loading branch information
szilard-nemeth committed Nov 26, 2019
1 parent 828ab40 commit ef950b0
Show file tree
Hide file tree
Showing 16 changed files with 125 additions and 12 deletions.
Expand Up @@ -357,6 +357,9 @@ public void allocate(ApplicationAttemptId appAttemptId,

response.setContainersFromPreviousAttempts(
allocation.getPreviousAttemptContainers());

response.setRejectedSchedulingRequests(allocation.getRejectedRequest());

}

private void handleInvalidResourceException(InvalidResourceRequestException e,
Expand Down
Expand Up @@ -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 {

Expand All @@ -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,
Expand All @@ -52,25 +53,26 @@ 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,
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
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;
Expand All @@ -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() {
Expand Down Expand Up @@ -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;
Expand Down
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -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,
Expand All @@ -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);
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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();
}
Expand Down
Expand Up @@ -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();
}
Expand Down
Expand Up @@ -951,7 +951,7 @@ public Allocation allocate(ApplicationAttemptId appAttemptId,
updatedNMTokens, null, null,
application.pullNewlyPromotedContainers(),
application.pullNewlyDemotedContainers(),
previousAttemptContainers);
previousAttemptContainers, null);
}

private List<MaxResourceValidationResult> validateResourceRequests(
Expand Down
Expand Up @@ -34,6 +34,7 @@
import java.util.Iterator;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicInteger;

/**
* <p>
Expand All @@ -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.
Expand Down Expand Up @@ -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();
}
}
Expand Up @@ -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;
}
}
Expand Down
Expand Up @@ -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;
Expand All @@ -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>());
Expand Down Expand Up @@ -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);
Expand Down
Expand Up @@ -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);

Expand Down Expand Up @@ -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);
Expand Down Expand Up @@ -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);
Expand Down Expand Up @@ -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);
Expand All @@ -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);
Expand All @@ -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);

Expand Down
Expand Up @@ -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.
Expand Down
Expand Up @@ -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);
Expand Down
Expand Up @@ -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));
Expand Down
Expand Up @@ -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);
Expand Down
Expand Up @@ -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);
Expand Down
Expand Up @@ -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
Expand Down
Expand Up @@ -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();
Expand Down

0 comments on commit ef950b0

Please sign in to comment.