YARN-7586. Application Placement should be done before ACL checks in ResourceManager. Contributed by Suma Shivaprasad.
This commit is contained in:
parent
9f1bdafedb
commit
f9f317b702
|
@ -48,6 +48,10 @@ import org.apache.hadoop.yarn.ipc.RPCUtil;
|
|||
import org.apache.hadoop.yarn.security.AccessRequest;
|
||||
import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
|
||||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.placement
|
||||
.ApplicationPlacementContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
|
||||
|
@ -361,7 +365,24 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|||
ApplicationSubmissionContext submissionContext, long submitTime,
|
||||
String user, boolean isRecovery, long startTime) throws YarnException {
|
||||
|
||||
ApplicationPlacementContext placementContext = null;
|
||||
|
||||
// We only do queue mapping when it's a new application
|
||||
if (!isRecovery) {
|
||||
try {
|
||||
// Do queue mapping
|
||||
placementContext = placeApplication(rmContext,
|
||||
submissionContext, user);
|
||||
replaceQueueFromPlacementContext(placementContext,
|
||||
submissionContext);
|
||||
} catch (YarnException e) {
|
||||
String msg = "Failed to place application " +
|
||||
submissionContext.getApplicationId() + " to queue and specified "
|
||||
+ "queue is invalid : " + submissionContext.getQueue();
|
||||
LOG.error(msg, e);
|
||||
throw e;
|
||||
}
|
||||
|
||||
// fail the submission if configured application timeout value is invalid
|
||||
RMServerUtils.validateApplicationTimeouts(
|
||||
submissionContext.getApplicationTimeouts());
|
||||
|
@ -413,7 +434,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|||
submissionContext.getQueue(),
|
||||
submissionContext, this.scheduler, this.masterService,
|
||||
submitTime, submissionContext.getApplicationType(),
|
||||
submissionContext.getApplicationTags(), amReqs, startTime);
|
||||
submissionContext.getApplicationTags(), amReqs, placementContext,
|
||||
startTime);
|
||||
// Concurrent app submissions with same applicationId will fail here
|
||||
// Concurrent app submissions with different applicationIds will not
|
||||
// influence each other
|
||||
|
@ -758,4 +780,40 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|||
+ "' with below exception:" + ex.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
ApplicationPlacementContext placeApplication(RMContext rmContext,
|
||||
ApplicationSubmissionContext context, String user) throws YarnException {
|
||||
ApplicationPlacementContext placementContext = null;
|
||||
PlacementManager placementManager = rmContext.getQueuePlacementManager();
|
||||
|
||||
if (placementManager != null) {
|
||||
placementContext = placementManager.placeApplication(context, user);
|
||||
} else{
|
||||
if ( context.getQueue() == null || context.getQueue().isEmpty()) {
|
||||
final String msg = "Queue Placement Manager is not set. Cannot place "
|
||||
+ "application : " + context.getApplicationId() + " to queue and "
|
||||
+ "specified queue is invalid " + context.getQueue();
|
||||
LOG.error(msg);
|
||||
throw new YarnException(msg);
|
||||
}
|
||||
}
|
||||
|
||||
return placementContext;
|
||||
}
|
||||
|
||||
void replaceQueueFromPlacementContext(
|
||||
ApplicationPlacementContext placementContext,
|
||||
ApplicationSubmissionContext context) {
|
||||
// Set it to ApplicationSubmissionContext
|
||||
//apply queue mapping only to new application submissions
|
||||
if (placementContext != null && !StringUtils.equalsIgnoreCase(
|
||||
context.getQueue(), placementContext.getQueue())) {
|
||||
LOG.info("Placed application=" + context.getApplicationId() +
|
||||
" to queue=" + placementContext.getQueue() + ", original queue="
|
||||
+ context
|
||||
.getQueue());
|
||||
context.setQueue(placementContext.getQueue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -72,8 +72,9 @@ public class PlacementManager {
|
|||
|
||||
// Failed to get where to place application
|
||||
if (null == placement && null == asc.getQueue()) {
|
||||
String msg = "Failed to get where to place application=" + asc
|
||||
.getApplicationId();
|
||||
String msg = "Failed to place application " +
|
||||
asc.getApplicationId() + " to queue and specified "
|
||||
+ "queue is invalid : " + asc.getQueue();
|
||||
LOG.error(msg);
|
||||
throw new YarnException(msg);
|
||||
}
|
||||
|
|
|
@ -43,6 +43,8 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.placement
|
||||
.ApplicationPlacementContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||
|
||||
|
@ -303,4 +305,10 @@ public interface RMApp extends EventHandler<RMAppEvent> {
|
|||
* @return True/False to confirm whether app is in final states
|
||||
*/
|
||||
boolean isAppInCompletedStates();
|
||||
|
||||
/**
|
||||
* Get the application -> queue placement context
|
||||
* @return ApplicationPlacementContext
|
||||
*/
|
||||
ApplicationPlacementContext getApplicationPlacementContext();
|
||||
}
|
||||
|
|
|
@ -67,7 +67,6 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
|
||||
|
@ -83,8 +82,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.blacklist.BlacklistManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.blacklist.DisabledBlacklistManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.blacklist.SimpleBlacklistManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
|
||||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.placement
|
||||
.ApplicationPlacementContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
|
||||
|
@ -204,6 +204,8 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
|
||||
private CallerContext callerContext;
|
||||
|
||||
private ApplicationPlacementContext placementContext;
|
||||
|
||||
Object transitionTodo;
|
||||
|
||||
private Priority applicationPriority;
|
||||
|
@ -417,7 +419,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
List<ResourceRequest> amReqs) {
|
||||
this(applicationId, rmContext, config, name, user, queue, submissionContext,
|
||||
scheduler, masterService, submitTime, applicationType, applicationTags,
|
||||
amReqs, -1);
|
||||
amReqs, null, -1);
|
||||
}
|
||||
|
||||
public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
|
||||
|
@ -425,7 +427,8 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
ApplicationSubmissionContext submissionContext, YarnScheduler scheduler,
|
||||
ApplicationMasterService masterService, long submitTime,
|
||||
String applicationType, Set<String> applicationTags,
|
||||
List<ResourceRequest> amReqs, long startTime) {
|
||||
List<ResourceRequest> amReqs, ApplicationPlacementContext
|
||||
placementContext, long startTime) {
|
||||
|
||||
this.systemClock = SystemClock.getInstance();
|
||||
|
||||
|
@ -484,6 +487,8 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
|
||||
this.callerContext = CallerContext.getCurrent();
|
||||
|
||||
this.placementContext = placementContext;
|
||||
|
||||
long localLogAggregationStatusTimeout =
|
||||
conf.getLong(YarnConfiguration.LOG_AGGREGATION_STATUS_TIME_OUT_MS,
|
||||
YarnConfiguration.DEFAULT_LOG_AGGREGATION_STATUS_TIME_OUT_MS);
|
||||
|
@ -1098,22 +1103,12 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
}
|
||||
}
|
||||
|
||||
ApplicationPlacementContext placementContext = null;
|
||||
try {
|
||||
placementContext = placeApplication(app.rmContext,
|
||||
app.submissionContext, app.user);
|
||||
} catch (Exception e) {
|
||||
String msg = "Failed to place application to queue :" + e.getMessage();
|
||||
app.diagnostics.append(msg);
|
||||
LOG.error(msg, e);
|
||||
}
|
||||
|
||||
// No existent attempts means the attempt associated with this app was not
|
||||
// started or started but not yet saved.
|
||||
if (app.attempts.isEmpty()) {
|
||||
app.scheduler.handle(
|
||||
new AppAddedSchedulerEvent(app.user, app.submissionContext, false,
|
||||
app.applicationPriority, placementContext));
|
||||
app.applicationPriority, app.placementContext));
|
||||
return RMAppState.SUBMITTED;
|
||||
}
|
||||
|
||||
|
@ -1121,7 +1116,7 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
// knows applications before AM or NM re-registers.
|
||||
app.scheduler.handle(
|
||||
new AppAddedSchedulerEvent(app.user, app.submissionContext, true,
|
||||
app.applicationPriority, placementContext));
|
||||
app.applicationPriority, app.placementContext));
|
||||
|
||||
// recover attempts
|
||||
app.recoverAppAttempts();
|
||||
|
@ -1137,20 +1132,9 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
RMAppTransition {
|
||||
@Override
|
||||
public void transition(RMAppImpl app, RMAppEvent event) {
|
||||
ApplicationPlacementContext placementContext = null;
|
||||
try {
|
||||
placementContext = placeApplication(app.rmContext,
|
||||
app.submissionContext, app.user);
|
||||
replaceQueueFromPlacementContext(placementContext,
|
||||
app.submissionContext);
|
||||
} catch (YarnException e) {
|
||||
String msg = "Failed to place application to queue :" + e.getMessage();
|
||||
app.diagnostics.append(msg);
|
||||
LOG.error(msg, e);
|
||||
}
|
||||
app.handler.handle(
|
||||
new AppAddedSchedulerEvent(app.user, app.submissionContext, false,
|
||||
app.applicationPriority, placementContext));
|
||||
app.applicationPriority, app.placementContext));
|
||||
// send the ATS create Event
|
||||
app.sendATSCreateEvent();
|
||||
}
|
||||
|
@ -1624,6 +1608,11 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
|| appState == RMAppState.KILLING;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationPlacementContext getApplicationPlacementContext() {
|
||||
return placementContext;
|
||||
}
|
||||
|
||||
public RMAppState getRecoveredFinalState() {
|
||||
return this.recoveredFinalState;
|
||||
}
|
||||
|
@ -2046,37 +2035,4 @@ public class RMAppImpl implements RMApp, Recoverable {
|
|||
this.submissionContext.setAMContainerSpec(null);
|
||||
this.submissionContext.setLogAggregationContext(null);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static ApplicationPlacementContext placeApplication(RMContext rmContext,
|
||||
ApplicationSubmissionContext context, String user) throws YarnException {
|
||||
|
||||
ApplicationPlacementContext placementContext = null;
|
||||
PlacementManager placementManager = rmContext.getQueuePlacementManager();
|
||||
|
||||
if (placementManager != null) {
|
||||
placementContext = placementManager.placeApplication(context, user);
|
||||
} else{
|
||||
LOG.error(
|
||||
"Queue Placement Manager is null. Cannot place application :" + " "
|
||||
+ context.getApplicationId() + " to queue ");
|
||||
}
|
||||
|
||||
return placementContext;
|
||||
}
|
||||
|
||||
static void replaceQueueFromPlacementContext(
|
||||
ApplicationPlacementContext placementContext,
|
||||
ApplicationSubmissionContext context) {
|
||||
// Set it to ApplicationSubmissionContext
|
||||
//apply queue mapping only to new application submissions
|
||||
if (placementContext != null && !StringUtils.equals(context.getQueue(),
|
||||
placementContext.getQueue())) {
|
||||
LOG.info("Placed application=" + context.getApplicationId() + " to queue="
|
||||
+ placementContext.getQueue() + ", original queue=" + context
|
||||
.getQueue());
|
||||
context.setQueue(placementContext.getQueue());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.yarn.server.resourcemanager;
|
||||
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.anyLong;
|
||||
import static org.mockito.Matchers.isA;
|
||||
|
@ -84,6 +85,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptI
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
|
||||
.CapacityScheduler;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
|
||||
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
||||
|
@ -110,6 +113,9 @@ public class TestAppManager{
|
|||
private Log LOG = LogFactory.getLog(TestAppManager.class);
|
||||
private static RMAppEventType appEventType = RMAppEventType.KILL;
|
||||
|
||||
private static String USER = "user_";
|
||||
private static String USER0 = USER + 0;
|
||||
|
||||
public synchronized RMAppEventType getAppEventType() {
|
||||
return appEventType;
|
||||
}
|
||||
|
@ -267,6 +273,70 @@ public class TestAppManager{
|
|||
setupDispatcher(rmContext, conf);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQueueSubmitWithACLsEnabledWithQueueMapping()
|
||||
throws IOException, YarnException, InterruptedException {
|
||||
YarnConfiguration conf = new YarnConfiguration();
|
||||
conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
|
||||
ResourceScheduler.class);
|
||||
conf.set(PREFIX + "root.queues", "default,test");
|
||||
|
||||
conf.setFloat(PREFIX + "root.default.capacity", 50.0f);
|
||||
conf.setFloat(PREFIX + "root.default.maximum-capacity", 100.0f);
|
||||
|
||||
conf.setFloat(PREFIX + "root.test.capacity", 50.0f);
|
||||
conf.setFloat(PREFIX + "root.test.maximum-capacity", 100.0f);
|
||||
|
||||
conf.set(PREFIX + "root.acl_submit_applications", " ");
|
||||
conf.set(PREFIX + "root.acl_administer_queue", " ");
|
||||
|
||||
conf.set(PREFIX + "root.default.acl_submit_applications", " ");
|
||||
conf.set(PREFIX + "root.default.acl_administer_queue", " ");
|
||||
|
||||
conf.set(PREFIX + "root.test.acl_submit_applications", "test");
|
||||
conf.set(PREFIX + "root.test.acl_administer_queue", "test");
|
||||
|
||||
conf.set(YarnConfiguration.YARN_ACL_ENABLE, "true");
|
||||
|
||||
// Setup a PlacementManager returns a new queue
|
||||
PlacementManager placementMgr = mock(PlacementManager.class);
|
||||
doAnswer(new Answer<ApplicationPlacementContext>() {
|
||||
|
||||
@Override
|
||||
public ApplicationPlacementContext answer(InvocationOnMock invocation)
|
||||
throws Throwable {
|
||||
return new ApplicationPlacementContext("test");
|
||||
}
|
||||
|
||||
}).when(placementMgr).placeApplication(
|
||||
any(ApplicationSubmissionContext.class), any(String.class));
|
||||
|
||||
asContext.setQueue("oldQueue");
|
||||
|
||||
MockRM newMockRM = new MockRM(conf);
|
||||
RMContext newMockRMContext = newMockRM.getRMContext();
|
||||
newMockRMContext.setQueuePlacementManager(placementMgr);
|
||||
ApplicationMasterService masterService =
|
||||
new ApplicationMasterService(newMockRMContext,
|
||||
newMockRMContext.getScheduler());
|
||||
|
||||
TestRMAppManager newAppMonitor = new TestRMAppManager(newMockRMContext,
|
||||
new ClientToAMTokenSecretManagerInRM(),
|
||||
newMockRMContext.getScheduler(), masterService,
|
||||
new ApplicationACLsManager(conf), conf);
|
||||
|
||||
//only user test has permission to submit to 'test' queue
|
||||
newAppMonitor.submitApplication(asContext, "test");
|
||||
|
||||
try {
|
||||
//should fail since user does not have permission to submit to queue
|
||||
// 'test'
|
||||
newAppMonitor.submitApplication(asContext, "test1");
|
||||
} catch(YarnException e) {
|
||||
assertTrue(e.getCause() instanceof AccessControlException);
|
||||
}
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() {
|
||||
setAppEventType(RMAppEventType.KILL);
|
||||
|
@ -328,7 +398,7 @@ public class TestAppManager{
|
|||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
if (e instanceof YarnException) {
|
||||
Assert.assertTrue(e.getCause() instanceof AccessControlException);
|
||||
assertTrue(e.getCause() instanceof AccessControlException);
|
||||
} else {
|
||||
Assert.fail("Yarn exception is expected : " + e.getMessage());
|
||||
}
|
||||
|
@ -718,7 +788,7 @@ public class TestAppManager{
|
|||
" Tokens are invalid.");
|
||||
} catch (YarnException e) {
|
||||
// Exception is expected
|
||||
Assert.assertTrue("The thrown exception is not" +
|
||||
assertTrue("The thrown exception is not" +
|
||||
" java.io.EOFException",
|
||||
e.getMessage().contains("java.io.EOFException"));
|
||||
}
|
||||
|
@ -778,22 +848,25 @@ public class TestAppManager{
|
|||
ApplicationId appId = MockApps.newAppID(0);
|
||||
asContext.setApplicationId(appId);
|
||||
RMApp appOrig = rmContext.getRMApps().get(appId);
|
||||
Assert.assertTrue("app name matches but shouldn't", "testApp1" != appOrig.getName());
|
||||
assertTrue("app name matches "
|
||||
+ "but shouldn't", "testApp1" != appOrig.getName());
|
||||
|
||||
// our testApp1 should be rejected and original app with same id should be left in place
|
||||
try {
|
||||
appMonitor.submitApplication(asContext, "test");
|
||||
Assert.fail("Exception is expected when applicationId is duplicate.");
|
||||
} catch (YarnException e) {
|
||||
Assert.assertTrue("The thrown exception is not the expectd one.",
|
||||
assertTrue("The thrown exception is not the expectd one.",
|
||||
e.getMessage().contains("Cannot add a duplicate!"));
|
||||
}
|
||||
|
||||
// make sure original app didn't get removed
|
||||
RMApp app = rmContext.getRMApps().get(appId);
|
||||
Assert.assertNotNull("app is null", app);
|
||||
Assert.assertEquals("app id doesn't match", appId, app.getApplicationId());
|
||||
Assert.assertEquals("app state doesn't match", RMAppState.FINISHED, app.getState());
|
||||
Assert.assertEquals("app id doesn't match",
|
||||
appId, app.getApplicationId());
|
||||
Assert.assertEquals("app state doesn't match",
|
||||
RMAppState.FINISHED, app.getState());
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
|
@ -811,7 +884,7 @@ public class TestAppManager{
|
|||
// Exception is expected
|
||||
// TODO Change this to assert the expected exception type - post YARN-142
|
||||
// sub-task related to specialized exceptions.
|
||||
Assert.assertTrue("The thrown exception is not" +
|
||||
assertTrue("The thrown exception is not" +
|
||||
" InvalidResourceRequestException",
|
||||
e.getMessage().contains("Invalid resource request"));
|
||||
}
|
||||
|
@ -844,16 +917,16 @@ public class TestAppManager{
|
|||
Assert.assertFalse(msg.contains("\r"));
|
||||
|
||||
String escaped = "\\n\\n\\r\\r";
|
||||
Assert.assertTrue(msg.contains("Multiline" + escaped +"AppName"));
|
||||
Assert.assertTrue(msg.contains("Multiline" + escaped +"UserName"));
|
||||
Assert.assertTrue(msg.contains("Multiline" + escaped +"QueueName"));
|
||||
Assert.assertTrue(msg.contains("submitTime=1000"));
|
||||
Assert.assertTrue(msg.contains("memorySeconds=16384"));
|
||||
Assert.assertTrue(msg.contains("vcoreSeconds=64"));
|
||||
Assert.assertTrue(msg.contains("preemptedAMContainers=1"));
|
||||
Assert.assertTrue(msg.contains("preemptedNonAMContainers=10"));
|
||||
Assert.assertTrue(msg.contains("preemptedResources=<memory:1234\\, vCores:56>"));
|
||||
Assert.assertTrue(msg.contains("applicationType=MAPREDUCE"));
|
||||
assertTrue(msg.contains("Multiline" + escaped +"AppName"));
|
||||
assertTrue(msg.contains("Multiline" + escaped +"UserName"));
|
||||
assertTrue(msg.contains("Multiline" + escaped +"QueueName"));
|
||||
assertTrue(msg.contains("submitTime=1000"));
|
||||
assertTrue(msg.contains("memorySeconds=16384"));
|
||||
assertTrue(msg.contains("vcoreSeconds=64"));
|
||||
assertTrue(msg.contains("preemptedAMContainers=1"));
|
||||
assertTrue(msg.contains("preemptedNonAMContainers=10"));
|
||||
assertTrue(msg.contains("preemptedResources=<memory:1234\\, vCores:56>"));
|
||||
assertTrue(msg.contains("applicationType=MAPREDUCE"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -952,4 +1025,5 @@ public class TestAppManager{
|
|||
}
|
||||
return cloneReqs;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -46,6 +46,8 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.placement
|
||||
.ApplicationPlacementContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
|
||||
|
@ -243,6 +245,11 @@ public abstract class MockAsm extends MockApps {
|
|||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationPlacementContext getApplicationPlacementContext() {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public CollectorInfo getCollectorInfo() {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
|
|
|
@ -45,6 +45,8 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPB
|
|||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport;
|
||||
import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.placement
|
||||
.ApplicationPlacementContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||
|
||||
|
@ -327,6 +329,11 @@ public class MockRMApp implements RMApp {
|
|||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationPlacementContext getApplicationPlacementContext() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CollectorInfo getCollectorInfo() {
|
||||
throw new UnsupportedOperationException("Not supported yet.");
|
||||
|
|
Loading…
Reference in New Issue