YARN-1493. Changed ResourceManager and Scheduler interfacing to recognize app-attempts separately from apps. Contributed by Jian He.

svn merge --ignore-ancestry -c 1554896 ../../trunk/


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1554898 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2014-01-02 20:21:03 +00:00
parent 4cf8f575fc
commit 8e135fc73c
38 changed files with 1261 additions and 896 deletions

View File

@ -64,8 +64,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
@ -105,8 +106,8 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
private Configuration conf; private Configuration conf;
private ResourceScheduler scheduler; private ResourceScheduler scheduler;
private Map<ApplicationAttemptId, String> appQueueMap = private Map<ApplicationId, String> appQueueMap =
new ConcurrentHashMap<ApplicationAttemptId, String>(); new ConcurrentHashMap<ApplicationId, String>();
private BufferedWriter jobRuntimeLogBW; private BufferedWriter jobRuntimeLogBW;
// Priority of the ResourceSchedulerWrapper shutdown hook. // Priority of the ResourceSchedulerWrapper shutdown hook.
@ -241,7 +242,7 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
(AppAttemptRemovedSchedulerEvent) schedulerEvent; (AppAttemptRemovedSchedulerEvent) schedulerEvent;
ApplicationAttemptId appAttemptId = ApplicationAttemptId appAttemptId =
appRemoveEvent.getApplicationAttemptID(); appRemoveEvent.getApplicationAttemptID();
String queue = appQueueMap.get(appAttemptId); String queue = appQueueMap.get(appAttemptId.getApplicationId());
SchedulerAppReport app = scheduler.getSchedulerAppInfo(appAttemptId); SchedulerAppReport app = scheduler.getSchedulerAppInfo(appAttemptId);
if (! app.getLiveContainers().isEmpty()) { // have 0 or 1 if (! app.getLiveContainers().isEmpty()) { // have 0 or 1
// should have one container which is AM container // should have one container which is AM container
@ -263,20 +264,18 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
schedulerHandleCounter.inc(); schedulerHandleCounter.inc();
schedulerHandleCounterMap.get(schedulerEvent.getType()).inc(); schedulerHandleCounterMap.get(schedulerEvent.getType()).inc();
if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED if (schedulerEvent.getType() == SchedulerEventType.APP_REMOVED
&& schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) { && schedulerEvent instanceof AppRemovedSchedulerEvent) {
SLSRunner.decreaseRemainingApps(); SLSRunner.decreaseRemainingApps();
AppAttemptRemovedSchedulerEvent appRemoveEvent = AppRemovedSchedulerEvent appRemoveEvent =
(AppAttemptRemovedSchedulerEvent) schedulerEvent; (AppRemovedSchedulerEvent) schedulerEvent;
ApplicationAttemptId appAttemptId = appQueueMap.remove(appRemoveEvent.getApplicationID());
appRemoveEvent.getApplicationAttemptID(); } else if (schedulerEvent.getType() == SchedulerEventType.APP_ADDED
appQueueMap.remove(appRemoveEvent.getApplicationAttemptID()); && schedulerEvent instanceof AppAddedSchedulerEvent) {
} else if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_ADDED AppAddedSchedulerEvent appAddEvent =
&& schedulerEvent instanceof AppAttemptAddedSchedulerEvent) { (AppAddedSchedulerEvent) schedulerEvent;
AppAttemptAddedSchedulerEvent appAddEvent =
(AppAttemptAddedSchedulerEvent) schedulerEvent;
String queueName = appAddEvent.getQueue(); String queueName = appAddEvent.getQueue();
appQueueMap.put(appAddEvent.getApplicationAttemptId(), queueName); appQueueMap.put(appAddEvent.getApplicationId(), queueName);
} }
} }
} }
@ -298,7 +297,9 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
continue; continue;
} }
String queue = appQueueMap.get(containerId.getApplicationAttemptId()); String queue =
appQueueMap.get(containerId.getApplicationAttemptId()
.getApplicationId());
int releasedMemory = 0, releasedVCores = 0; int releasedMemory = 0, releasedVCores = 0;
if (status.getExitStatus() == ContainerExitStatus.SUCCESS) { if (status.getExitStatus() == ContainerExitStatus.SUCCESS) {
for (RMContainer rmc : app.getLiveContainers()) { for (RMContainer rmc : app.getLiveContainers()) {
@ -330,7 +331,7 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
// update queue information // update queue information
Resource pendingResource = Resources.createResource(0, 0); Resource pendingResource = Resources.createResource(0, 0);
Resource allocatedResource = Resources.createResource(0, 0); Resource allocatedResource = Resources.createResource(0, 0);
String queueName = appQueueMap.get(attemptId); String queueName = appQueueMap.get(attemptId.getApplicationId());
// container requested // container requested
for (ResourceRequest request : resourceRequests) { for (ResourceRequest request : resourceRequests) {
if (request.getResourceName().equals(ResourceRequest.ANY)) { if (request.getResourceName().equals(ResourceRequest.ANY)) {

View File

@ -176,6 +176,9 @@ Release 2.4.0 - UNRELEASED
YARN-1541. Changed ResourceManager to invalidate ApplicationMaster host/port YARN-1541. Changed ResourceManager to invalidate ApplicationMaster host/port
information once an AM crashes. (Jian He via vinodkv) information once an AM crashes. (Jian He via vinodkv)
YARN-1493. Changed ResourceManager and Scheduler interfacing to recognize
app-attempts separately from apps. (Jian He via vinodkv)
OPTIMIZATIONS OPTIMIZATIONS
BUG FIXES BUG FIXES

View File

@ -24,9 +24,11 @@ public enum RMAppEventType {
RECOVER, RECOVER,
KILL, KILL,
// Source: RMAppAttempt // Source: Scheduler
APP_REJECTED, APP_REJECTED,
APP_ACCEPTED, APP_ACCEPTED,
// Source: RMAppAttempt
ATTEMPT_REGISTERED, ATTEMPT_REGISTERED,
ATTEMPT_UNREGISTERED, ATTEMPT_UNREGISTERED,
ATTEMPT_FINISHED, // Will send the final state ATTEMPT_FINISHED, // Will send the final state

View File

@ -66,6 +66,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptS
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.state.InvalidStateTransitonException; import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
import org.apache.hadoop.yarn.state.MultipleArcTransition; import org.apache.hadoop.yarn.state.MultipleArcTransition;
@ -136,7 +138,7 @@ public class RMAppImpl implements RMApp, Recoverable {
.addTransition(RMAppState.NEW, RMAppState.NEW_SAVING, .addTransition(RMAppState.NEW, RMAppState.NEW_SAVING,
RMAppEventType.START, new RMAppNewlySavingTransition()) RMAppEventType.START, new RMAppNewlySavingTransition())
.addTransition(RMAppState.NEW, EnumSet.of(RMAppState.SUBMITTED, .addTransition(RMAppState.NEW, EnumSet.of(RMAppState.SUBMITTED,
RMAppState.RUNNING, RMAppState.FINISHED, RMAppState.FAILED, RMAppState.ACCEPTED, RMAppState.FINISHED, RMAppState.FAILED,
RMAppState.KILLED, RMAppState.FINAL_SAVING), RMAppState.KILLED, RMAppState.FINAL_SAVING),
RMAppEventType.RECOVER, new RMAppRecoveredTransition()) RMAppEventType.RECOVER, new RMAppRecoveredTransition())
.addTransition(RMAppState.NEW, RMAppState.FINAL_SAVING, RMAppEventType.KILL, .addTransition(RMAppState.NEW, RMAppState.FINAL_SAVING, RMAppEventType.KILL,
@ -151,7 +153,7 @@ public class RMAppImpl implements RMApp, Recoverable {
.addTransition(RMAppState.NEW_SAVING, RMAppState.NEW_SAVING, .addTransition(RMAppState.NEW_SAVING, RMAppState.NEW_SAVING,
RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition()) RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
.addTransition(RMAppState.NEW_SAVING, RMAppState.SUBMITTED, .addTransition(RMAppState.NEW_SAVING, RMAppState.SUBMITTED,
RMAppEventType.APP_NEW_SAVED, new StartAppAttemptTransition()) RMAppEventType.APP_NEW_SAVED, new AddApplicationToSchedulerTransition())
.addTransition(RMAppState.NEW_SAVING, RMAppState.FINAL_SAVING, .addTransition(RMAppState.NEW_SAVING, RMAppState.FINAL_SAVING,
RMAppEventType.KILL, RMAppEventType.KILL,
new FinalSavingTransition( new FinalSavingTransition(
@ -169,9 +171,12 @@ public class RMAppImpl implements RMApp, Recoverable {
new FinalSavingTransition( new FinalSavingTransition(
new AppRejectedTransition(), RMAppState.FAILED)) new AppRejectedTransition(), RMAppState.FAILED))
.addTransition(RMAppState.SUBMITTED, RMAppState.ACCEPTED, .addTransition(RMAppState.SUBMITTED, RMAppState.ACCEPTED,
RMAppEventType.APP_ACCEPTED) RMAppEventType.APP_ACCEPTED, new StartAppAttemptTransition())
.addTransition(RMAppState.SUBMITTED, RMAppState.KILLING, .addTransition(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING,
RMAppEventType.KILL,new KillAttemptTransition()) RMAppEventType.KILL,
new FinalSavingTransition(
new AppKilledTransition(), RMAppState.KILLED))
// Transitions from ACCEPTED state // Transitions from ACCEPTED state
.addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED, .addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
@ -179,11 +184,22 @@ public class RMAppImpl implements RMApp, Recoverable {
.addTransition(RMAppState.ACCEPTED, RMAppState.RUNNING, .addTransition(RMAppState.ACCEPTED, RMAppState.RUNNING,
RMAppEventType.ATTEMPT_REGISTERED) RMAppEventType.ATTEMPT_REGISTERED)
.addTransition(RMAppState.ACCEPTED, .addTransition(RMAppState.ACCEPTED,
EnumSet.of(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING), EnumSet.of(RMAppState.ACCEPTED, RMAppState.FINAL_SAVING),
// ACCEPTED state is possible to receive ATTEMPT_FAILED event because
// RMAppRecoveredTransition is returning ACCEPTED state directly and
// waiting for the previous AM to exit.
RMAppEventType.ATTEMPT_FAILED, RMAppEventType.ATTEMPT_FAILED,
new AttemptFailedTransition(RMAppState.SUBMITTED)) new AttemptFailedTransition(RMAppState.ACCEPTED))
.addTransition(RMAppState.ACCEPTED, RMAppState.KILLING, .addTransition(RMAppState.ACCEPTED, RMAppState.FINAL_SAVING,
RMAppEventType.KILL,new KillAttemptTransition()) RMAppEventType.KILL,
new FinalSavingTransition(
new AppKilledTransition(), RMAppState.KILLED))
// ACCECPTED state can once again receive APP_ACCEPTED event, because on
// recovery the app returns ACCEPTED state and the app once again go
// through the scheduler and triggers one more APP_ACCEPTED event at
// ACCEPTED state.
.addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
RMAppEventType.APP_ACCEPTED)
// Transitions from RUNNING state // Transitions from RUNNING state
.addTransition(RMAppState.RUNNING, RMAppState.RUNNING, .addTransition(RMAppState.RUNNING, RMAppState.RUNNING,
@ -197,9 +213,9 @@ public class RMAppImpl implements RMApp, Recoverable {
// UnManagedAM directly jumps to finished // UnManagedAM directly jumps to finished
RMAppEventType.ATTEMPT_FINISHED, FINISHED_TRANSITION) RMAppEventType.ATTEMPT_FINISHED, FINISHED_TRANSITION)
.addTransition(RMAppState.RUNNING, .addTransition(RMAppState.RUNNING,
EnumSet.of(RMAppState.SUBMITTED, RMAppState.FINAL_SAVING), EnumSet.of(RMAppState.ACCEPTED, RMAppState.FINAL_SAVING),
RMAppEventType.ATTEMPT_FAILED, RMAppEventType.ATTEMPT_FAILED,
new AttemptFailedTransition(RMAppState.SUBMITTED)) new AttemptFailedTransition(RMAppState.ACCEPTED))
.addTransition(RMAppState.RUNNING, RMAppState.KILLING, .addTransition(RMAppState.RUNNING, RMAppState.KILLING,
RMAppEventType.KILL, new KillAttemptTransition()) RMAppEventType.KILL, new KillAttemptTransition())
@ -641,7 +657,7 @@ public class RMAppImpl implements RMApp, Recoverable {
ApplicationAttemptId.newInstance(applicationId, attempts.size() + 1); ApplicationAttemptId.newInstance(applicationId, attempts.size() + 1);
RMAppAttempt attempt = RMAppAttempt attempt =
new RMAppAttemptImpl(appAttemptId, rmContext, scheduler, masterService, new RMAppAttemptImpl(appAttemptId, rmContext, scheduler, masterService,
submissionContext, conf, user); submissionContext, conf);
attempts.put(appAttemptId, attempt); attempts.put(appAttemptId, attempt);
currentAttempt = attempt; currentAttempt = attempt;
if(startAttempt) { if(startAttempt) {
@ -695,29 +711,46 @@ public class RMAppImpl implements RMApp, Recoverable {
return app.recoveredFinalState; return app.recoveredFinalState;
} }
// Notify scheduler about the app on recovery
new AddApplicationToSchedulerTransition().transition(app, event);
// No existent attempts means the attempt associated with this app was not // No existent attempts means the attempt associated with this app was not
// started or started but not yet saved. // started or started but not yet saved.
if (app.attempts.isEmpty()) { if (app.attempts.isEmpty()) {
app.createNewAttempt(true);
return RMAppState.SUBMITTED; return RMAppState.SUBMITTED;
} }
return RMAppState.RUNNING; // YARN-1507 is saving the application state after the application is
// accepted. So after YARN-1507, an app is saved meaning it is accepted.
// Thus we return ACCECPTED state on recovery.
return RMAppState.ACCEPTED;
}
}
private static final class AddApplicationToSchedulerTransition extends
RMAppTransition {
@SuppressWarnings("unchecked")
@Override
public void transition(RMAppImpl app, RMAppEvent event) {
if (event instanceof RMAppNewSavedEvent) {
RMAppNewSavedEvent storeEvent = (RMAppNewSavedEvent) event;
// For HA this exception needs to be handled by giving up
// master status if we got fenced
if (((RMAppNewSavedEvent) event).getStoredException() != null) {
LOG.error(
"Failed to store application: " + storeEvent.getApplicationId(),
storeEvent.getStoredException());
ExitUtil.terminate(1, storeEvent.getStoredException());
}
}
app.handler.handle(new AppAddedSchedulerEvent(app.applicationId,
app.submissionContext.getQueue(), app.user));
} }
} }
private static final class StartAppAttemptTransition extends RMAppTransition { private static final class StartAppAttemptTransition extends RMAppTransition {
@Override @Override
public void transition(RMAppImpl app, RMAppEvent event) { public void transition(RMAppImpl app, RMAppEvent event) {
RMAppNewSavedEvent storeEvent = (RMAppNewSavedEvent) event;
if (storeEvent.getStoredException() != null) {
// For HA this exception needs to be handled by giving up
// master status if we got fenced
LOG.error(
"Failed to store application: " + storeEvent.getApplicationId(),
storeEvent.getStoredException());
ExitUtil.terminate(1, storeEvent.getStoredException());
}
app.createNewAttempt(true); app.createNewAttempt(true);
}; };
} }
@ -965,6 +998,8 @@ public class RMAppImpl implements RMApp, Recoverable {
if (app.finishTime == 0 ) { if (app.finishTime == 0 ) {
app.finishTime = System.currentTimeMillis(); app.finishTime = System.currentTimeMillis();
} }
app.handler.handle(new AppRemovedSchedulerEvent(app.applicationId, app
.getState()));
app.handler.handle( app.handler.handle(
new RMAppManagerEvent(app.applicationId, new RMAppManagerEvent(app.applicationId,
RMAppManagerEventType.APP_COMPLETED)); RMAppManagerEventType.APP_COMPLETED));
@ -993,7 +1028,6 @@ public class RMAppImpl implements RMApp, Recoverable {
return RMAppState.FINAL_SAVING; return RMAppState.FINAL_SAVING;
} }
} }
} }
@Override @Override

View File

@ -45,8 +45,7 @@ public enum RMAppAttemptEventType {
ATTEMPT_UPDATE_SAVED, ATTEMPT_UPDATE_SAVED,
// Source: Scheduler // Source: Scheduler
APP_REJECTED, ATTEMPT_ADDED,
APP_ACCEPTED,
// Source: RMAttemptImpl.recover // Source: RMAttemptImpl.recover
RECOVER RECOVER

View File

@ -75,13 +75,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFinishedAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFinishedAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStatusupdateEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStatusupdateEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
@ -150,7 +148,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
private final StringBuilder diagnostics = new StringBuilder(); private final StringBuilder diagnostics = new StringBuilder();
private Configuration conf; private Configuration conf;
private String user;
private static final ExpiredTransition EXPIRED_TRANSITION = private static final ExpiredTransition EXPIRED_TRANSITION =
new ExpiredTransition(); new ExpiredTransition();
@ -186,14 +183,10 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
RMAppAttemptEventType.RECOVER, new AttemptRecoveredTransition()) RMAppAttemptEventType.RECOVER, new AttemptRecoveredTransition())
// Transitions from SUBMITTED state // Transitions from SUBMITTED state
.addTransition(RMAppAttemptState.SUBMITTED, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.APP_REJECTED,
new FinalSavingTransition(new AppRejectedTransition(),
RMAppAttemptState.FAILED))
.addTransition(RMAppAttemptState.SUBMITTED, .addTransition(RMAppAttemptState.SUBMITTED,
EnumSet.of(RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING, EnumSet.of(RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING,
RMAppAttemptState.SCHEDULED), RMAppAttemptState.SCHEDULED),
RMAppAttemptEventType.APP_ACCEPTED, RMAppAttemptEventType.ATTEMPT_ADDED,
new ScheduleTransition()) new ScheduleTransition())
.addTransition(RMAppAttemptState.SUBMITTED, RMAppAttemptState.FINAL_SAVING, .addTransition(RMAppAttemptState.SUBMITTED, RMAppAttemptState.FINAL_SAVING,
RMAppAttemptEventType.KILL, RMAppAttemptEventType.KILL,
@ -380,8 +373,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
.addTransition( .addTransition(
RMAppAttemptState.KILLED, RMAppAttemptState.KILLED,
RMAppAttemptState.KILLED, RMAppAttemptState.KILLED,
EnumSet.of(RMAppAttemptEventType.APP_ACCEPTED, EnumSet.of(RMAppAttemptEventType.ATTEMPT_ADDED,
RMAppAttemptEventType.APP_REJECTED,
RMAppAttemptEventType.EXPIRE, RMAppAttemptEventType.EXPIRE,
RMAppAttemptEventType.LAUNCHED, RMAppAttemptEventType.LAUNCHED,
RMAppAttemptEventType.LAUNCH_FAILED, RMAppAttemptEventType.LAUNCH_FAILED,
@ -398,7 +390,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
RMContext rmContext, YarnScheduler scheduler, RMContext rmContext, YarnScheduler scheduler,
ApplicationMasterService masterService, ApplicationMasterService masterService,
ApplicationSubmissionContext submissionContext, ApplicationSubmissionContext submissionContext,
Configuration conf, String user) { Configuration conf) {
this.conf = conf; this.conf = conf;
this.applicationAttemptId = appAttemptId; this.applicationAttemptId = appAttemptId;
this.rmContext = rmContext; this.rmContext = rmContext;
@ -414,7 +406,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
this.proxiedTrackingUrl = generateProxyUriWithScheme(null); this.proxiedTrackingUrl = generateProxyUriWithScheme(null);
this.stateMachine = stateMachineFactory.make(this); this.stateMachine = stateMachineFactory.make(this);
this.user = user;
} }
@Override @Override
@ -750,35 +741,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
appAttempt.rmContext.getAMRMTokenSecretManager()); appAttempt.rmContext.getAMRMTokenSecretManager());
// Add the applicationAttempt to the scheduler // Add the applicationAttempt to the scheduler
appAttempt.eventHandler.handle( appAttempt.eventHandler.handle(new AppAttemptAddedSchedulerEvent(
new AppAttemptAddedSchedulerEvent(appAttempt.applicationAttemptId, appAttempt.applicationAttemptId));
appAttempt.submissionContext.getQueue(), appAttempt.user));
}
}
private static final class AppRejectedTransition extends BaseTransition {
@Override
public void transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) {
RMAppAttemptRejectedEvent rejectedEvent = (RMAppAttemptRejectedEvent) event;
// Tell the AMS. Unregister from the ApplicationMasterService
appAttempt.masterService
.unregisterAttempt(appAttempt.applicationAttemptId);
// Save the diagnostic message
String message = rejectedEvent.getMessage();
appAttempt.diagnostics.append(message);
// Send the rejection event to app
appAttempt.eventHandler.handle(
new RMAppRejectedEvent(
rejectedEvent.getApplicationAttemptId().getApplicationId(),
message)
);
appAttempt.removeCredentials(appAttempt);
} }
} }
@ -794,11 +758,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
public RMAppAttemptState transition(RMAppAttemptImpl appAttempt, public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) { RMAppAttemptEvent event) {
if (!appAttempt.submissionContext.getUnmanagedAM()) { if (!appAttempt.submissionContext.getUnmanagedAM()) {
// Send the acceptance to the app
appAttempt.eventHandler.handle(new RMAppEvent(event
.getApplicationAttemptId().getApplicationId(),
RMAppEventType.APP_ACCEPTED));
// Request a container for the AM. // Request a container for the AM.
ResourceRequest request = ResourceRequest request =
BuilderUtils.newResourceRequest( BuilderUtils.newResourceRequest(
@ -918,11 +877,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
FinalApplicationStatus finalStatus = null; FinalApplicationStatus finalStatus = null;
switch (event.getType()) { switch (event.getType()) {
case APP_REJECTED:
RMAppAttemptRejectedEvent rejectedEvent =
(RMAppAttemptRejectedEvent) event;
diags = rejectedEvent.getMessage();
break;
case LAUNCH_FAILED: case LAUNCH_FAILED:
RMAppAttemptLaunchFailedEvent launchFaileEvent = RMAppAttemptLaunchFailedEvent launchFaileEvent =
(RMAppAttemptLaunchFailedEvent) event; (RMAppAttemptLaunchFailedEvent) event;
@ -1091,16 +1045,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
public void transition(RMAppAttemptImpl appAttempt, public void transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) { RMAppAttemptEvent event) {
appAttempt.checkAttemptStoreError(event); appAttempt.checkAttemptStoreError(event);
// Send the acceptance to the app
// Ideally this should have been done when the scheduler accepted the app.
// But its here because until the attempt is saved the client should not
// launch the unmanaged AM. Client waits for the app status to be accepted
// before doing so. So we have to delay the accepted state until we have
// completed storing the attempt
appAttempt.eventHandler.handle(new RMAppEvent(event
.getApplicationAttemptId().getApplicationId(),
RMAppEventType.APP_ACCEPTED));
super.transition(appAttempt, event); super.transition(appAttempt, event);
} }
} }

View File

@ -56,7 +56,7 @@ public class ActiveUsersManager {
* @param user application user * @param user application user
* @param applicationId activated application * @param applicationId activated application
*/ */
@Lock({Queue.class, SchedulerApplication.class}) @Lock({Queue.class, SchedulerApplicationAttempt.class})
synchronized public void activateApplication( synchronized public void activateApplication(
String user, ApplicationId applicationId) { String user, ApplicationId applicationId) {
Set<ApplicationId> userApps = usersApplications.get(user); Set<ApplicationId> userApps = usersApplications.get(user);
@ -79,7 +79,7 @@ public class ActiveUsersManager {
* @param user application user * @param user application user
* @param applicationId deactivated application * @param applicationId deactivated application
*/ */
@Lock({Queue.class, SchedulerApplication.class}) @Lock({Queue.class, SchedulerApplicationAttempt.class})
synchronized public void deactivateApplication( synchronized public void deactivateApplication(
String user, ApplicationId applicationId) { String user, ApplicationId applicationId) {
Set<ApplicationId> userApps = usersApplications.get(user); Set<ApplicationId> userApps = usersApplications.get(user);
@ -102,7 +102,7 @@ public class ActiveUsersManager {
* resource requests. * resource requests.
* @return number of active users * @return number of active users
*/ */
@Lock({Queue.class, SchedulerApplication.class}) @Lock({Queue.class, SchedulerApplicationAttempt.class})
synchronized public int getNumActiveUsers() { synchronized public int getNumActiveUsers() {
return activeUsers; return activeUsers;
} }

View File

@ -36,7 +36,7 @@ public class SchedulerAppReport {
private final Collection<RMContainer> reserved; private final Collection<RMContainer> reserved;
private final boolean pending; private final boolean pending;
public SchedulerAppReport(SchedulerApplication app) { public SchedulerAppReport(SchedulerApplicationAttempt app) {
this.live = app.getLiveContainers(); this.live = app.getLiveContainers();
this.reserved = app.getReservedContainers(); this.reserved = app.getReservedContainers();
this.pending = app.isPending(); this.pending = app.isPending();

View File

@ -22,7 +22,7 @@ import org.apache.commons.logging.Log;
public class SchedulerAppUtils { public class SchedulerAppUtils {
public static boolean isBlacklisted(SchedulerApplication application, public static boolean isBlacklisted(SchedulerApplicationAttempt application,
SchedulerNode node, Log LOG) { SchedulerNode node, Log LOG) {
if (application.isBlacklisted(node.getNodeName())) { if (application.isBlacklisted(node.getNodeName())) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {

View File

@ -17,393 +17,26 @@
*/ */
package org.apache.hadoop.yarn.server.resourcemanager.scheduler; package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
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.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReservedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.collect.HashMultiset;
import com.google.common.collect.Multiset;
/**
* Represents an application attempt from the viewpoint of the scheduler.
* Each running app attempt in the RM corresponds to one instance
* of this class.
*/
@Private @Private
@Unstable @Unstable
public abstract class SchedulerApplication { public class SchedulerApplication {
private static final Log LOG = LogFactory.getLog(SchedulerApplication.class); private final Queue queue;
private final String user;
protected final AppSchedulingInfo appSchedulingInfo; public SchedulerApplication(Queue queue, String user) {
protected final Map<ContainerId, RMContainer> liveContainers =
new HashMap<ContainerId, RMContainer>();
protected final Map<Priority, Map<NodeId, RMContainer>> reservedContainers =
new HashMap<Priority, Map<NodeId, RMContainer>>();
private final Multiset<Priority> reReservations = HashMultiset.create();
protected final Resource currentReservation = Resource.newInstance(0, 0);
private Resource resourceLimit = Resource.newInstance(0, 0);
protected final Resource currentConsumption = Resource.newInstance(0, 0);
protected List<RMContainer> newlyAllocatedContainers =
new ArrayList<RMContainer>();
/**
* Count how many times the application has been given an opportunity
* to schedule a task at each priority. Each time the scheduler
* asks the application for a task at this priority, it is incremented,
* and each time the application successfully schedules a task, it
* is reset to 0.
*/
Multiset<Priority> schedulingOpportunities = HashMultiset.create();
// Time of the last container scheduled at the current allowed level
protected Map<Priority, Long> lastScheduledContainer =
new HashMap<Priority, Long>();
protected final Queue queue;
protected boolean isStopped = false;
protected final RMContext rmContext;
public SchedulerApplication(ApplicationAttemptId applicationAttemptId,
String user, Queue queue, ActiveUsersManager activeUsersManager,
RMContext rmContext) {
this.rmContext = rmContext;
this.appSchedulingInfo =
new AppSchedulingInfo(applicationAttemptId, user, queue,
activeUsersManager);
this.queue = queue; this.queue = queue;
} this.user = user;
/**
* Get the live containers of the application.
* @return live containers of the application
*/
public synchronized Collection<RMContainer> getLiveContainers() {
return new ArrayList<RMContainer>(liveContainers.values());
}
/**
* Is this application pending?
* @return true if it is else false.
*/
public boolean isPending() {
return appSchedulingInfo.isPending();
}
/**
* Get {@link ApplicationAttemptId} of the application master.
* @return <code>ApplicationAttemptId</code> of the application master
*/
public ApplicationAttemptId getApplicationAttemptId() {
return appSchedulingInfo.getApplicationAttemptId();
}
public ApplicationId getApplicationId() {
return appSchedulingInfo.getApplicationId();
}
public String getUser() {
return appSchedulingInfo.getUser();
}
public Map<String, ResourceRequest> getResourceRequests(Priority priority) {
return appSchedulingInfo.getResourceRequests(priority);
}
public int getNewContainerId() {
return appSchedulingInfo.getNewContainerId();
}
public Collection<Priority> getPriorities() {
return appSchedulingInfo.getPriorities();
}
public ResourceRequest getResourceRequest(Priority priority, String resourceName) {
return this.appSchedulingInfo.getResourceRequest(priority, resourceName);
}
public synchronized int getTotalRequiredResources(Priority priority) {
return getResourceRequest(priority, ResourceRequest.ANY).getNumContainers();
}
public Resource getResource(Priority priority) {
return appSchedulingInfo.getResource(priority);
}
public String getQueueName() {
return appSchedulingInfo.getQueueName();
}
public synchronized RMContainer getRMContainer(ContainerId id) {
return liveContainers.get(id);
}
protected synchronized void resetReReservations(Priority priority) {
reReservations.setCount(priority, 0);
}
protected synchronized void addReReservation(Priority priority) {
reReservations.add(priority);
}
public synchronized int getReReservations(Priority priority) {
return reReservations.count(priority);
}
/**
* Get total current reservations.
* Used only by unit tests
* @return total current reservations
*/
@Stable
@Private
public synchronized Resource getCurrentReservation() {
return currentReservation;
} }
public Queue getQueue() { public Queue getQueue() {
return queue; return queue;
} }
public synchronized void updateResourceRequests( public String getUser() {
List<ResourceRequest> requests) { return user;
if (!isStopped) {
appSchedulingInfo.updateResourceRequests(requests);
} }
}
public synchronized void stop(RMAppAttemptState rmAppAttemptFinalState) {
// Cleanup all scheduling information
isStopped = true;
appSchedulingInfo.stop(rmAppAttemptFinalState);
}
public synchronized boolean isStopped() {
return isStopped;
}
/**
* Get the list of reserved containers
* @return All of the reserved containers.
*/
public synchronized List<RMContainer> getReservedContainers() {
List<RMContainer> reservedContainers = new ArrayList<RMContainer>();
for (Map.Entry<Priority, Map<NodeId, RMContainer>> e :
this.reservedContainers.entrySet()) {
reservedContainers.addAll(e.getValue().values());
}
return reservedContainers;
}
public synchronized RMContainer reserve(SchedulerNode node, Priority priority,
RMContainer rmContainer, Container container) {
// Create RMContainer if necessary
if (rmContainer == null) {
rmContainer =
new RMContainerImpl(container, getApplicationAttemptId(),
node.getNodeID(), rmContext.getDispatcher().getEventHandler(),
rmContext.getContainerAllocationExpirer());
Resources.addTo(currentReservation, container.getResource());
// Reset the re-reservation count
resetReReservations(priority);
} else {
// Note down the re-reservation
addReReservation(priority);
}
rmContainer.handle(new RMContainerReservedEvent(container.getId(),
container.getResource(), node.getNodeID(), priority));
Map<NodeId, RMContainer> reservedContainers =
this.reservedContainers.get(priority);
if (reservedContainers == null) {
reservedContainers = new HashMap<NodeId, RMContainer>();
this.reservedContainers.put(priority, reservedContainers);
}
reservedContainers.put(node.getNodeID(), rmContainer);
LOG.info("Application " + getApplicationId()
+ " reserved container " + rmContainer
+ " on node " + node + ", currently has " + reservedContainers.size()
+ " at priority " + priority
+ "; currentReservation " + currentReservation.getMemory());
return rmContainer;
}
/**
* Has the application reserved the given <code>node</code> at the
* given <code>priority</code>?
* @param node node to be checked
* @param priority priority of reserved container
* @return true is reserved, false if not
*/
public synchronized boolean isReserved(SchedulerNode node, Priority priority) {
Map<NodeId, RMContainer> reservedContainers =
this.reservedContainers.get(priority);
if (reservedContainers != null) {
return reservedContainers.containsKey(node.getNodeID());
}
return false;
}
public synchronized void setHeadroom(Resource globalLimit) {
this.resourceLimit = globalLimit;
}
/**
* Get available headroom in terms of resources for the application's user.
* @return available resource headroom
*/
public synchronized Resource getHeadroom() {
// Corner case to deal with applications being slightly over-limit
if (resourceLimit.getMemory() < 0) {
resourceLimit.setMemory(0);
}
return resourceLimit;
}
public synchronized int getNumReservedContainers(Priority priority) {
Map<NodeId, RMContainer> reservedContainers =
this.reservedContainers.get(priority);
return (reservedContainers == null) ? 0 : reservedContainers.size();
}
@SuppressWarnings("unchecked")
public synchronized void containerLaunchedOnNode(ContainerId containerId,
NodeId nodeId) {
// Inform the container
RMContainer rmContainer = getRMContainer(containerId);
if (rmContainer == null) {
// Some unknown container sneaked into the system. Kill it.
rmContext.getDispatcher().getEventHandler()
.handle(new RMNodeCleanContainerEvent(nodeId, containerId));
return;
}
rmContainer.handle(new RMContainerEvent(containerId,
RMContainerEventType.LAUNCHED));
}
public synchronized void showRequests() {
if (LOG.isDebugEnabled()) {
for (Priority priority : getPriorities()) {
Map<String, ResourceRequest> requests = getResourceRequests(priority);
if (requests != null) {
LOG.debug("showRequests:" + " application=" + getApplicationId() +
" headRoom=" + getHeadroom() +
" currentConsumption=" + currentConsumption.getMemory());
for (ResourceRequest request : requests.values()) {
LOG.debug("showRequests:" + " application=" + getApplicationId()
+ " request=" + request);
}
}
}
}
}
public Resource getCurrentConsumption() {
return currentConsumption;
}
public synchronized List<Container> pullNewlyAllocatedContainers() {
List<Container> returnContainerList = new ArrayList<Container>(
newlyAllocatedContainers.size());
for (RMContainer rmContainer : newlyAllocatedContainers) {
rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(),
RMContainerEventType.ACQUIRED));
returnContainerList.add(rmContainer.getContainer());
}
newlyAllocatedContainers.clear();
return returnContainerList;
}
public synchronized void updateBlacklist(
List<String> blacklistAdditions, List<String> blacklistRemovals) {
if (!isStopped) {
this.appSchedulingInfo.updateBlacklist(
blacklistAdditions, blacklistRemovals);
}
}
public boolean isBlacklisted(String resourceName) {
return this.appSchedulingInfo.isBlacklisted(resourceName);
}
public synchronized void addSchedulingOpportunity(Priority priority) {
schedulingOpportunities.setCount(priority,
schedulingOpportunities.count(priority) + 1);
}
public synchronized void subtractSchedulingOpportunity(Priority priority) {
int count = schedulingOpportunities.count(priority) - 1;
this.schedulingOpportunities.setCount(priority, Math.max(count, 0));
}
/**
* Return the number of times the application has been given an opportunity
* to schedule a task at the given priority since the last time it
* successfully did so.
*/
public synchronized int getSchedulingOpportunities(Priority priority) {
return schedulingOpportunities.count(priority);
}
/**
* Should be called when an application has successfully scheduled a container,
* or when the scheduling locality threshold is relaxed.
* Reset various internal counters which affect delay scheduling
*
* @param priority The priority of the container scheduled.
*/
public synchronized void resetSchedulingOpportunities(Priority priority) {
resetSchedulingOpportunities(priority, System.currentTimeMillis());
}
// used for continuous scheduling
public synchronized void resetSchedulingOpportunities(Priority priority,
long currentTimeMs) {
lastScheduledContainer.put(priority, currentTimeMs);
schedulingOpportunities.setCount(priority, 0);
}
public synchronized ApplicationResourceUsageReport getResourceUsageReport() {
return ApplicationResourceUsageReport.newInstance(liveContainers.size(),
reservedContainers.size(), Resources.clone(currentConsumption),
Resources.clone(currentReservation),
Resources.add(currentConsumption, currentReservation));
}
} }

View File

@ -0,0 +1,410 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
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.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReservedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.collect.HashMultiset;
import com.google.common.collect.Multiset;
/**
* Represents an application attempt from the viewpoint of the scheduler.
* Each running app attempt in the RM corresponds to one instance
* of this class.
*/
@Private
@Unstable
public abstract class SchedulerApplicationAttempt {
private static final Log LOG = LogFactory
.getLog(SchedulerApplicationAttempt.class);
protected final AppSchedulingInfo appSchedulingInfo;
protected final Map<ContainerId, RMContainer> liveContainers =
new HashMap<ContainerId, RMContainer>();
protected final Map<Priority, Map<NodeId, RMContainer>> reservedContainers =
new HashMap<Priority, Map<NodeId, RMContainer>>();
private final Multiset<Priority> reReservations = HashMultiset.create();
protected final Resource currentReservation = Resource.newInstance(0, 0);
private Resource resourceLimit = Resource.newInstance(0, 0);
protected final Resource currentConsumption = Resource.newInstance(0, 0);
protected List<RMContainer> newlyAllocatedContainers =
new ArrayList<RMContainer>();
/**
* Count how many times the application has been given an opportunity
* to schedule a task at each priority. Each time the scheduler
* asks the application for a task at this priority, it is incremented,
* and each time the application successfully schedules a task, it
* is reset to 0.
*/
Multiset<Priority> schedulingOpportunities = HashMultiset.create();
// Time of the last container scheduled at the current allowed level
protected Map<Priority, Long> lastScheduledContainer =
new HashMap<Priority, Long>();
protected final Queue queue;
protected boolean isStopped = false;
protected final RMContext rmContext;
public SchedulerApplicationAttempt(ApplicationAttemptId applicationAttemptId,
String user, Queue queue, ActiveUsersManager activeUsersManager,
RMContext rmContext) {
this.rmContext = rmContext;
this.appSchedulingInfo =
new AppSchedulingInfo(applicationAttemptId, user, queue,
activeUsersManager);
this.queue = queue;
}
/**
* Get the live containers of the application.
* @return live containers of the application
*/
public synchronized Collection<RMContainer> getLiveContainers() {
return new ArrayList<RMContainer>(liveContainers.values());
}
/**
* Is this application pending?
* @return true if it is else false.
*/
public boolean isPending() {
return appSchedulingInfo.isPending();
}
/**
* Get {@link ApplicationAttemptId} of the application master.
* @return <code>ApplicationAttemptId</code> of the application master
*/
public ApplicationAttemptId getApplicationAttemptId() {
return appSchedulingInfo.getApplicationAttemptId();
}
public ApplicationId getApplicationId() {
return appSchedulingInfo.getApplicationId();
}
public String getUser() {
return appSchedulingInfo.getUser();
}
public Map<String, ResourceRequest> getResourceRequests(Priority priority) {
return appSchedulingInfo.getResourceRequests(priority);
}
public int getNewContainerId() {
return appSchedulingInfo.getNewContainerId();
}
public Collection<Priority> getPriorities() {
return appSchedulingInfo.getPriorities();
}
public ResourceRequest getResourceRequest(Priority priority, String resourceName) {
return this.appSchedulingInfo.getResourceRequest(priority, resourceName);
}
public synchronized int getTotalRequiredResources(Priority priority) {
return getResourceRequest(priority, ResourceRequest.ANY).getNumContainers();
}
public Resource getResource(Priority priority) {
return appSchedulingInfo.getResource(priority);
}
public String getQueueName() {
return appSchedulingInfo.getQueueName();
}
public synchronized RMContainer getRMContainer(ContainerId id) {
return liveContainers.get(id);
}
protected synchronized void resetReReservations(Priority priority) {
reReservations.setCount(priority, 0);
}
protected synchronized void addReReservation(Priority priority) {
reReservations.add(priority);
}
public synchronized int getReReservations(Priority priority) {
return reReservations.count(priority);
}
/**
* Get total current reservations.
* Used only by unit tests
* @return total current reservations
*/
@Stable
@Private
public synchronized Resource getCurrentReservation() {
return currentReservation;
}
public Queue getQueue() {
return queue;
}
public synchronized void updateResourceRequests(
List<ResourceRequest> requests) {
if (!isStopped) {
appSchedulingInfo.updateResourceRequests(requests);
}
}
public synchronized void stop(RMAppAttemptState rmAppAttemptFinalState) {
// Cleanup all scheduling information
isStopped = true;
appSchedulingInfo.stop(rmAppAttemptFinalState);
}
public synchronized boolean isStopped() {
return isStopped;
}
/**
* Get the list of reserved containers
* @return All of the reserved containers.
*/
public synchronized List<RMContainer> getReservedContainers() {
List<RMContainer> reservedContainers = new ArrayList<RMContainer>();
for (Map.Entry<Priority, Map<NodeId, RMContainer>> e :
this.reservedContainers.entrySet()) {
reservedContainers.addAll(e.getValue().values());
}
return reservedContainers;
}
public synchronized RMContainer reserve(SchedulerNode node, Priority priority,
RMContainer rmContainer, Container container) {
// Create RMContainer if necessary
if (rmContainer == null) {
rmContainer =
new RMContainerImpl(container, getApplicationAttemptId(),
node.getNodeID(), rmContext.getDispatcher().getEventHandler(),
rmContext.getContainerAllocationExpirer());
Resources.addTo(currentReservation, container.getResource());
// Reset the re-reservation count
resetReReservations(priority);
} else {
// Note down the re-reservation
addReReservation(priority);
}
rmContainer.handle(new RMContainerReservedEvent(container.getId(),
container.getResource(), node.getNodeID(), priority));
Map<NodeId, RMContainer> reservedContainers =
this.reservedContainers.get(priority);
if (reservedContainers == null) {
reservedContainers = new HashMap<NodeId, RMContainer>();
this.reservedContainers.put(priority, reservedContainers);
}
reservedContainers.put(node.getNodeID(), rmContainer);
LOG.info("Application " + getApplicationId()
+ " reserved container " + rmContainer
+ " on node " + node + ", currently has " + reservedContainers.size()
+ " at priority " + priority
+ "; currentReservation " + currentReservation.getMemory());
return rmContainer;
}
/**
* Has the application reserved the given <code>node</code> at the
* given <code>priority</code>?
* @param node node to be checked
* @param priority priority of reserved container
* @return true is reserved, false if not
*/
public synchronized boolean isReserved(SchedulerNode node, Priority priority) {
Map<NodeId, RMContainer> reservedContainers =
this.reservedContainers.get(priority);
if (reservedContainers != null) {
return reservedContainers.containsKey(node.getNodeID());
}
return false;
}
public synchronized void setHeadroom(Resource globalLimit) {
this.resourceLimit = globalLimit;
}
/**
* Get available headroom in terms of resources for the application's user.
* @return available resource headroom
*/
public synchronized Resource getHeadroom() {
// Corner case to deal with applications being slightly over-limit
if (resourceLimit.getMemory() < 0) {
resourceLimit.setMemory(0);
}
return resourceLimit;
}
public synchronized int getNumReservedContainers(Priority priority) {
Map<NodeId, RMContainer> reservedContainers =
this.reservedContainers.get(priority);
return (reservedContainers == null) ? 0 : reservedContainers.size();
}
@SuppressWarnings("unchecked")
public synchronized void containerLaunchedOnNode(ContainerId containerId,
NodeId nodeId) {
// Inform the container
RMContainer rmContainer = getRMContainer(containerId);
if (rmContainer == null) {
// Some unknown container sneaked into the system. Kill it.
rmContext.getDispatcher().getEventHandler()
.handle(new RMNodeCleanContainerEvent(nodeId, containerId));
return;
}
rmContainer.handle(new RMContainerEvent(containerId,
RMContainerEventType.LAUNCHED));
}
public synchronized void showRequests() {
if (LOG.isDebugEnabled()) {
for (Priority priority : getPriorities()) {
Map<String, ResourceRequest> requests = getResourceRequests(priority);
if (requests != null) {
LOG.debug("showRequests:" + " application=" + getApplicationId() +
" headRoom=" + getHeadroom() +
" currentConsumption=" + currentConsumption.getMemory());
for (ResourceRequest request : requests.values()) {
LOG.debug("showRequests:" + " application=" + getApplicationId()
+ " request=" + request);
}
}
}
}
}
public Resource getCurrentConsumption() {
return currentConsumption;
}
public synchronized List<Container> pullNewlyAllocatedContainers() {
List<Container> returnContainerList = new ArrayList<Container>(
newlyAllocatedContainers.size());
for (RMContainer rmContainer : newlyAllocatedContainers) {
rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(),
RMContainerEventType.ACQUIRED));
returnContainerList.add(rmContainer.getContainer());
}
newlyAllocatedContainers.clear();
return returnContainerList;
}
public synchronized void updateBlacklist(
List<String> blacklistAdditions, List<String> blacklistRemovals) {
if (!isStopped) {
this.appSchedulingInfo.updateBlacklist(
blacklistAdditions, blacklistRemovals);
}
}
public boolean isBlacklisted(String resourceName) {
return this.appSchedulingInfo.isBlacklisted(resourceName);
}
public synchronized void addSchedulingOpportunity(Priority priority) {
schedulingOpportunities.setCount(priority,
schedulingOpportunities.count(priority) + 1);
}
public synchronized void subtractSchedulingOpportunity(Priority priority) {
int count = schedulingOpportunities.count(priority) - 1;
this.schedulingOpportunities.setCount(priority, Math.max(count, 0));
}
/**
* Return the number of times the application has been given an opportunity
* to schedule a task at the given priority since the last time it
* successfully did so.
*/
public synchronized int getSchedulingOpportunities(Priority priority) {
return schedulingOpportunities.count(priority);
}
/**
* Should be called when an application has successfully scheduled a container,
* or when the scheduling locality threshold is relaxed.
* Reset various internal counters which affect delay scheduling
*
* @param priority The priority of the container scheduled.
*/
public synchronized void resetSchedulingOpportunities(Priority priority) {
resetSchedulingOpportunities(priority, System.currentTimeMillis());
}
// used for continuous scheduling
public synchronized void resetSchedulingOpportunities(Priority priority,
long currentTimeMs) {
lastScheduledContainer.put(priority, currentTimeMs);
schedulingOpportunities.setCount(priority, 0);
}
public synchronized ApplicationResourceUsageReport getResourceUsageReport() {
return ApplicationResourceUsageReport.newInstance(liveContainers.size(),
reservedContainers.size(), Resources.clone(currentConsumption),
Resources.clone(currentReservation),
Resources.add(currentConsumption, currentReservation));
}
}

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueACL;
@ -35,7 +36,6 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
@ -155,20 +155,31 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
/** /**
* Submit a new application to the queue. * Submit a new application to the queue.
* @param application application being submitted * @param applicationId the applicationId of the application being submitted
* @param user user who submitted the application * @param user user who submitted the application
* @param queue queue to which the application is submitted * @param queue queue to which the application is submitted
*/ */
public void submitApplication(FiCaSchedulerApp application, String user, public void submitApplication(ApplicationId applicationId, String user,
String queue) String queue) throws AccessControlException;
throws AccessControlException;
/**
* Submit an application attempt to the queue.
*/
public void submitApplicationAttempt(FiCaSchedulerApp application,
String userName);
/** /**
* An application submitted to this queue has finished. * An application submitted to this queue has finished.
* @param application * @param applicationId
* @param queue application queue * @param user user who submitted the application
*/ */
public void finishApplication(FiCaSchedulerApp application, String queue); public void finishApplication(ApplicationId applicationId, String user);
/**
* An application attempt submitted to this queue has finished.
*/
public void finishApplicationAttempt(FiCaSchedulerApp application,
String queue);
/** /**
* Assign containers to applications in the queue or it's children (if any). * Assign containers to applications in the queue or it's children (if any).

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
@ -53,10 +54,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@ -65,14 +69,16 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
@ -185,7 +191,11 @@ public class CapacityScheduler
private Resource maximumAllocation; private Resource maximumAllocation;
@VisibleForTesting @VisibleForTesting
protected Map<ApplicationAttemptId, FiCaSchedulerApp> applications = protected Map<ApplicationId, SchedulerApplication> applications =
new ConcurrentHashMap<ApplicationId, SchedulerApplication>();
@VisibleForTesting
protected Map<ApplicationAttemptId, FiCaSchedulerApp> appAttempts =
new ConcurrentHashMap<ApplicationAttemptId, FiCaSchedulerApp>(); new ConcurrentHashMap<ApplicationAttemptId, FiCaSchedulerApp>();
private boolean initialized = false; private boolean initialized = false;
@ -416,60 +426,83 @@ public class CapacityScheduler
return queues.get(queueName); return queues.get(queueName);
} }
private synchronized void private synchronized void addApplication(ApplicationId applicationId,
addApplicationAttempt(ApplicationAttemptId applicationAttemptId,
String queueName, String user) { String queueName, String user) {
// santiy checks.
// Sanity checks
CSQueue queue = getQueue(queueName); CSQueue queue = getQueue(queueName);
if (queue == null) { if (queue == null) {
String message = "Application " + applicationAttemptId + String message = "Application " + applicationId +
" submitted by user " + user + " to unknown queue: " + queueName; " submitted by user " + user + " to unknown queue: " + queueName;
this.rmContext.getDispatcher().getEventHandler().handle( this.rmContext.getDispatcher().getEventHandler()
new RMAppAttemptRejectedEvent(applicationAttemptId, message)); .handle(new RMAppRejectedEvent(applicationId, message));
return; return;
} }
if (!(queue instanceof LeafQueue)) { if (!(queue instanceof LeafQueue)) {
String message = "Application " + applicationAttemptId + String message = "Application " + applicationId +
" submitted by user " + user + " to non-leaf queue: " + queueName; " submitted by user " + user + " to non-leaf queue: " + queueName;
this.rmContext.getDispatcher().getEventHandler().handle( this.rmContext.getDispatcher().getEventHandler()
new RMAppAttemptRejectedEvent(applicationAttemptId, message)); .handle(new RMAppRejectedEvent(applicationId, message));
return; return;
} }
// TODO: Fix store
FiCaSchedulerApp SchedulerApp =
new FiCaSchedulerApp(applicationAttemptId, user, queue,
queue.getActiveUsersManager(), rmContext);
// Submit to the queue // Submit to the queue
try { try {
queue.submitApplication(SchedulerApp, user, queueName); queue.submitApplication(applicationId, user, queueName);
} catch (AccessControlException ace) { } catch (AccessControlException ace) {
LOG.info("Failed to submit application " + applicationAttemptId + LOG.info("Failed to submit application " + applicationId + " to queue "
" to queue " + queueName + " from user " + user, ace); + queueName + " from user " + user, ace);
this.rmContext.getDispatcher().getEventHandler().handle( this.rmContext.getDispatcher().getEventHandler()
new RMAppAttemptRejectedEvent(applicationAttemptId, .handle(new RMAppRejectedEvent(applicationId, ace.toString()));
ace.toString()));
return; return;
} }
SchedulerApplication application =
new SchedulerApplication(queue, user);
applications.put(applicationId, application);
LOG.info("Accepted application " + applicationId + " from user: " + user
+ ", in queue: " + queueName);
rmContext.getDispatcher().getEventHandler()
.handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
}
applications.put(applicationAttemptId, SchedulerApp); private synchronized void addApplicationAttempt(
ApplicationAttemptId applicationAttemptId) {
LOG.info("Application Submission: " + applicationAttemptId + SchedulerApplication application =
", user: " + user + applications.get(applicationAttemptId.getApplicationId());
" queue: " + queue + CSQueue queue = (CSQueue) application.getQueue();
", currently active: " + applications.size());
FiCaSchedulerApp SchedulerApp =
new FiCaSchedulerApp(applicationAttemptId, application.getUser(),
queue, queue.getActiveUsersManager(), rmContext);
appAttempts.put(applicationAttemptId, SchedulerApp);
queue.submitApplicationAttempt(SchedulerApp, application.getUser());
LOG.info("Added Application Attempt " + applicationAttemptId
+ " to scheduler from user " + application.getUser() + " in queue "
+ queue.getQueueName());
rmContext.getDispatcher().getEventHandler().handle( rmContext.getDispatcher().getEventHandler().handle(
new RMAppAttemptEvent(applicationAttemptId, new RMAppAttemptEvent(applicationAttemptId,
RMAppAttemptEventType.APP_ACCEPTED)); RMAppAttemptEventType.ATTEMPT_ADDED));
}
private synchronized void doneApplication(ApplicationId applicationId,
RMAppState finalState) {
SchedulerApplication application = applications.get(applicationId);
if (application == null){
// The AppRemovedSchedulerEvent maybe sent on recovery for completed apps.
return;
}
CSQueue queue = (CSQueue) application.getQueue();
if (!(queue instanceof LeafQueue)) {
LOG.error("Cannot finish application " + "from non-leaf queue: "
+ queue.getQueueName());
} else {
queue.finishApplication(applicationId, application.getUser());
}
applications.remove(applicationId);
} }
private synchronized void doneApplicationAttempt( private synchronized void doneApplicationAttempt(
ApplicationAttemptId applicationAttemptId, ApplicationAttemptId applicationAttemptId,
RMAppAttemptState rmAppAttemptFinalState) { RMAppAttemptState rmAppAttemptFinalState) {
LOG.info("Application " + applicationAttemptId + " is done." + LOG.info("Application Attempt " + applicationAttemptId + " is done." +
" finalState=" + rmAppAttemptFinalState); " finalState=" + rmAppAttemptFinalState);
FiCaSchedulerApp application = getApplication(applicationAttemptId); FiCaSchedulerApp application = getApplication(applicationAttemptId);
@ -509,11 +542,11 @@ public class CapacityScheduler
LOG.error("Cannot finish application " + "from non-leaf queue: " LOG.error("Cannot finish application " + "from non-leaf queue: "
+ queueName); + queueName);
} else { } else {
queue.finishApplication(application, queue.getQueueName()); queue.finishApplicationAttempt(application, queue.getQueueName());
} }
// Remove from our data-structure // Remove from our data-structure
applications.remove(applicationAttemptId); appAttempts.remove(applicationAttemptId);
} }
private static final Allocation EMPTY_ALLOCATION = private static final Allocation EMPTY_ALLOCATION =
@ -740,12 +773,25 @@ public class CapacityScheduler
nodeUpdate(nodeUpdatedEvent.getRMNode()); nodeUpdate(nodeUpdatedEvent.getRMNode());
} }
break; break;
case APP_ADDED:
{
AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
addApplication(appAddedEvent.getApplicationId(),
appAddedEvent.getQueue(), appAddedEvent.getUser());
}
break;
case APP_REMOVED:
{
AppRemovedSchedulerEvent appRemovedEvent = (AppRemovedSchedulerEvent)event;
doneApplication(appRemovedEvent.getApplicationID(),
appRemovedEvent.getFinalState());
}
break;
case APP_ATTEMPT_ADDED: case APP_ATTEMPT_ADDED:
{ {
AppAttemptAddedSchedulerEvent appAttemptAddedEvent = AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
(AppAttemptAddedSchedulerEvent) event; (AppAttemptAddedSchedulerEvent) event;
addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(), addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId());
appAttemptAddedEvent.getQueue(), appAttemptAddedEvent.getUser());
} }
break; break;
case APP_ATTEMPT_REMOVED: case APP_ATTEMPT_REMOVED:
@ -854,7 +900,7 @@ public class CapacityScheduler
@Lock(Lock.NoLock.class) @Lock(Lock.NoLock.class)
FiCaSchedulerApp getApplication(ApplicationAttemptId applicationAttemptId) { FiCaSchedulerApp getApplication(ApplicationAttemptId applicationAttemptId) {
return applications.get(applicationAttemptId); return appAttempts.get(applicationAttemptId);
} }
@Override @Override
@ -912,7 +958,7 @@ public class CapacityScheduler
LOG.debug("PREEMPT_CONTAINER: application:" + aid.toString() + LOG.debug("PREEMPT_CONTAINER: application:" + aid.toString() +
" container: " + cont.toString()); " container: " + cont.toString());
} }
FiCaSchedulerApp app = applications.get(aid); FiCaSchedulerApp app = appAttempts.get(aid);
if (app != null) { if (app != null) {
app.addPreemptContainer(cont.getContainerId()); app.addPreemptContainer(cont.getContainerId());
} }

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
@ -59,7 +60,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@ -99,7 +99,7 @@ public class LeafQueue implements CSQueue {
private volatile int numContainers; private volatile int numContainers;
Set<FiCaSchedulerApp> activeApplications; Set<FiCaSchedulerApp> activeApplications;
Map<ApplicationAttemptId, FiCaSchedulerApp> applicationsMap = Map<ApplicationAttemptId, FiCaSchedulerApp> applicationAttemptMap =
new HashMap<ApplicationAttemptId, FiCaSchedulerApp>(); new HashMap<ApplicationAttemptId, FiCaSchedulerApp>();
Set<FiCaSchedulerApp> pendingApplications; Set<FiCaSchedulerApp> pendingApplications;
@ -635,7 +635,22 @@ public class LeafQueue implements CSQueue {
} }
@Override @Override
public void submitApplication(FiCaSchedulerApp application, String userName, public void submitApplicationAttempt(FiCaSchedulerApp application,
String userName) {
// Careful! Locking order is important!
synchronized (this) {
User user = getUser(userName);
// Add the attempt to our data-structures
addApplicationAttempt(application, user);
}
int attemptId = application.getApplicationAttemptId().getAttemptId();
metrics.submitApp(userName, attemptId);
getParent().submitApplicationAttempt(application, userName);
}
@Override
public void submitApplication(ApplicationId applicationId, String userName,
String queue) throws AccessControlException { String queue) throws AccessControlException {
// Careful! Locking order is important! // Careful! Locking order is important!
@ -653,8 +668,7 @@ public class LeafQueue implements CSQueue {
// Check if the queue is accepting jobs // Check if the queue is accepting jobs
if (getState() != QueueState.RUNNING) { if (getState() != QueueState.RUNNING) {
String msg = "Queue " + getQueuePath() + String msg = "Queue " + getQueuePath() +
" is STOPPED. Cannot accept submission of application: " + " is STOPPED. Cannot accept submission of application: " + applicationId;
application.getApplicationId();
LOG.info(msg); LOG.info(msg);
throw new AccessControlException(msg); throw new AccessControlException(msg);
} }
@ -663,8 +677,7 @@ public class LeafQueue implements CSQueue {
if (getNumApplications() >= getMaxApplications()) { if (getNumApplications() >= getMaxApplications()) {
String msg = "Queue " + getQueuePath() + String msg = "Queue " + getQueuePath() +
" already has " + getNumApplications() + " applications," + " already has " + getNumApplications() + " applications," +
" cannot accept submission of application: " + " cannot accept submission of application: " + applicationId;
application.getApplicationId();
LOG.info(msg); LOG.info(msg);
throw new AccessControlException(msg); throw new AccessControlException(msg);
} }
@ -675,26 +688,18 @@ public class LeafQueue implements CSQueue {
String msg = "Queue " + getQueuePath() + String msg = "Queue " + getQueuePath() +
" already has " + user.getTotalApplications() + " already has " + user.getTotalApplications() +
" applications from user " + userName + " applications from user " + userName +
" cannot accept submission of application: " + " cannot accept submission of application: " + applicationId;
application.getApplicationId();
LOG.info(msg); LOG.info(msg);
throw new AccessControlException(msg); throw new AccessControlException(msg);
} }
// Add the application to our data-structures
addApplication(application, user);
} }
int attemptId = application.getApplicationAttemptId().getAttemptId();
metrics.submitApp(userName, attemptId);
// Inform the parent queue // Inform the parent queue
try { try {
getParent().submitApplication(application, userName, queue); getParent().submitApplication(applicationId, userName, queue);
} catch (AccessControlException ace) { } catch (AccessControlException ace) {
LOG.info("Failed to submit application to parent-queue: " + LOG.info("Failed to submit application to parent-queue: " +
getParent().getQueuePath(), ace); getParent().getQueuePath(), ace);
removeApplication(application, user);
throw ace; throw ace;
} }
} }
@ -722,11 +727,11 @@ public class LeafQueue implements CSQueue {
} }
} }
private synchronized void addApplication(FiCaSchedulerApp application, User user) { private synchronized void addApplicationAttempt(FiCaSchedulerApp application, User user) {
// Accept // Accept
user.submitApplication(); user.submitApplication();
pendingApplications.add(application); pendingApplications.add(application);
applicationsMap.put(application.getApplicationAttemptId(), application); applicationAttemptMap.put(application.getApplicationAttemptId(), application);
// Activate applications // Activate applications
activateApplications(); activateApplications();
@ -742,22 +747,28 @@ public class LeafQueue implements CSQueue {
} }
@Override @Override
public void finishApplication(FiCaSchedulerApp application, String queue) { public void finishApplication(ApplicationId application, String user) {
// Inform the activeUsersManager
activeUsersManager.deactivateApplication(user, application);
// Inform the parent queue
getParent().finishApplication(application, user);
}
@Override
public void finishApplicationAttempt(FiCaSchedulerApp application, String queue) {
// Careful! Locking order is important! // Careful! Locking order is important!
synchronized (this) { synchronized (this) {
removeApplication(application, getUser(application.getUser())); removeApplicationAttempt(application, getUser(application.getUser()));
}
getParent().finishApplicationAttempt(application, queue);
} }
// Inform the parent queue public synchronized void removeApplicationAttempt(FiCaSchedulerApp application, User user) {
getParent().finishApplication(application, queue);
}
public synchronized void removeApplication(FiCaSchedulerApp application, User user) {
boolean wasActive = activeApplications.remove(application); boolean wasActive = activeApplications.remove(application);
if (!wasActive) { if (!wasActive) {
pendingApplications.remove(application); pendingApplications.remove(application);
} }
applicationsMap.remove(application.getApplicationAttemptId()); applicationAttemptMap.remove(application.getApplicationAttemptId());
user.finishApplication(wasActive); user.finishApplication(wasActive);
if (user.getTotalApplications() == 0) { if (user.getTotalApplications() == 0) {
@ -767,12 +778,6 @@ public class LeafQueue implements CSQueue {
// Check if we can activate more applications // Check if we can activate more applications
activateApplications(); activateApplications();
// Inform the activeUsersManager
synchronized (application) {
activeUsersManager.deactivateApplication(
application.getUser(), application.getApplicationId());
}
LOG.info("Application removed -" + LOG.info("Application removed -" +
" appId: " + application.getApplicationId() + " appId: " + application.getApplicationId() +
" user: " + application.getUser() + " user: " + application.getUser() +
@ -786,7 +791,7 @@ public class LeafQueue implements CSQueue {
private synchronized FiCaSchedulerApp getApplication( private synchronized FiCaSchedulerApp getApplication(
ApplicationAttemptId applicationAttemptId) { ApplicationAttemptId applicationAttemptId) {
return applicationsMap.get(applicationAttemptId); return applicationAttemptMap.get(applicationAttemptId);
} }
private static final CSAssignment NULL_ASSIGNMENT = private static final CSAssignment NULL_ASSIGNMENT =

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueACL;
@ -51,7 +52,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@ -442,7 +442,7 @@ public class ParentQueue implements CSQueue {
} }
@Override @Override
public void submitApplication(FiCaSchedulerApp application, String user, public void submitApplication(ApplicationId applicationId, String user,
String queue) throws AccessControlException { String queue) throws AccessControlException {
synchronized (this) { synchronized (this) {
@ -455,57 +455,70 @@ public class ParentQueue implements CSQueue {
if (state != QueueState.RUNNING) { if (state != QueueState.RUNNING) {
throw new AccessControlException("Queue " + getQueuePath() + throw new AccessControlException("Queue " + getQueuePath() +
" is STOPPED. Cannot accept submission of application: " + " is STOPPED. Cannot accept submission of application: " +
application.getApplicationId()); applicationId);
} }
addApplication(application, user); addApplication(applicationId, user);
} }
// Inform the parent queue // Inform the parent queue
if (parent != null) { if (parent != null) {
try { try {
parent.submitApplication(application, user, queue); parent.submitApplication(applicationId, user, queue);
} catch (AccessControlException ace) { } catch (AccessControlException ace) {
LOG.info("Failed to submit application to parent-queue: " + LOG.info("Failed to submit application to parent-queue: " +
parent.getQueuePath(), ace); parent.getQueuePath(), ace);
removeApplication(application, user); removeApplication(applicationId, user);
throw ace; throw ace;
} }
} }
} }
private synchronized void addApplication(FiCaSchedulerApp application,
@Override
public void submitApplicationAttempt(FiCaSchedulerApp application,
String userName) {
// submit attempt logic.
}
@Override
public void finishApplicationAttempt(FiCaSchedulerApp application,
String queue) {
// finish attempt logic.
}
private synchronized void addApplication(ApplicationId applicationId,
String user) { String user) {
++numApplications; ++numApplications;
LOG.info("Application added -" + LOG.info("Application added -" +
" appId: " + application.getApplicationId() + " appId: " + applicationId +
" user: " + user + " user: " + user +
" leaf-queue of parent: " + getQueueName() + " leaf-queue of parent: " + getQueueName() +
" #applications: " + getNumApplications()); " #applications: " + getNumApplications());
} }
@Override @Override
public void finishApplication(FiCaSchedulerApp application, String queue) { public void finishApplication(ApplicationId application, String user) {
synchronized (this) { synchronized (this) {
removeApplication(application, application.getUser()); removeApplication(application, user);
} }
// Inform the parent queue // Inform the parent queue
if (parent != null) { if (parent != null) {
parent.finishApplication(application, queue); parent.finishApplication(application, user);
} }
} }
public synchronized void removeApplication(FiCaSchedulerApp application, public synchronized void removeApplication(ApplicationId applicationId,
String user) { String user) {
--numApplications; --numApplications;
LOG.info("Application removed -" + LOG.info("Application removed -" +
" appId: " + application.getApplicationId() + " appId: " + applicationId +
" user: " + user + " user: " + user +
" leaf-queue of parent: " + getQueueName() + " leaf-queue of parent: " + getQueueName() +
" #applications: " + getNumApplications()); " #applications: " + getNumApplications());

View File

@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@ -57,7 +57,7 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
*/ */
@Private @Private
@Unstable @Unstable
public class FiCaSchedulerApp extends SchedulerApplication { public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
private static final Log LOG = LogFactory.getLog(FiCaSchedulerApp.class); private static final Log LOG = LogFactory.getLog(FiCaSchedulerApp.class);

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
@ -206,7 +206,7 @@ public class FiCaSchedulerNode extends SchedulerNode {
} }
public synchronized void reserveResource( public synchronized void reserveResource(
SchedulerApplication application, Priority priority, SchedulerApplicationAttempt application, Priority priority,
RMContainer reservedContainer) { RMContainer reservedContainer) {
// Check if it's already reserved // Check if it's already reserved
if (this.reservedContainer != null) { if (this.reservedContainer != null) {
@ -241,7 +241,7 @@ public class FiCaSchedulerNode extends SchedulerNode {
} }
public synchronized void unreserveResource( public synchronized void unreserveResource(
SchedulerApplication application) { SchedulerApplicationAttempt application) {
// adding NP checks as this can now be called for preemption // adding NP checks as this can now be called for preemption
if (reservedContainer != null if (reservedContainer != null

View File

@ -16,22 +16,34 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event; package org.apache.hadoop.yarn.server.resourcemanager.scheduler.event;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
public class RMAppAttemptRejectedEvent extends RMAppAttemptEvent { public class AppAddedSchedulerEvent extends SchedulerEvent {
private final String message; private final ApplicationId applicationId;
private final String queue;
private final String user;
public RMAppAttemptRejectedEvent(ApplicationAttemptId appAttemptId, String message) { public AppAddedSchedulerEvent(
super(appAttemptId, RMAppAttemptEventType.APP_REJECTED); ApplicationId applicationId, String queue, String user) {
this.message = message; super(SchedulerEventType.APP_ADDED);
this.applicationId = applicationId;
this.queue = queue;
this.user = user;
} }
public String getMessage() { public ApplicationId getApplicationId() {
return this.message; return applicationId;
} }
public String getQueue() {
return queue;
}
public String getUser() {
return user;
}
} }

View File

@ -23,27 +23,14 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
public class AppAttemptAddedSchedulerEvent extends SchedulerEvent { public class AppAttemptAddedSchedulerEvent extends SchedulerEvent {
private final ApplicationAttemptId applicationAttemptId; private final ApplicationAttemptId applicationAttemptId;
private final String queue;
private final String user;
public AppAttemptAddedSchedulerEvent( public AppAttemptAddedSchedulerEvent(
ApplicationAttemptId applicationAttemptId, String queue, String user) { ApplicationAttemptId applicationAttemptId) {
super(SchedulerEventType.APP_ATTEMPT_ADDED); super(SchedulerEventType.APP_ATTEMPT_ADDED);
this.applicationAttemptId = applicationAttemptId; this.applicationAttemptId = applicationAttemptId;
this.queue = queue;
this.user = user;
} }
public ApplicationAttemptId getApplicationAttemptId() { public ApplicationAttemptId getApplicationAttemptId() {
return applicationAttemptId; return applicationAttemptId;
} }
public String getQueue() {
return queue;
}
public String getUser() {
return user;
}
} }

View File

@ -0,0 +1,43 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.event;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
public class AppRemovedSchedulerEvent extends SchedulerEvent {
private final ApplicationId applicationId;
private final RMAppState finalState;
public AppRemovedSchedulerEvent(ApplicationId applicationId,
RMAppState finalState) {
super(SchedulerEventType.APP_REMOVED);
this.applicationId = applicationId;
this.finalState = finalState;
}
public ApplicationId getApplicationID() {
return this.applicationId;
}
public RMAppState getFinalState() {
return this.finalState;
}
}

View File

@ -25,6 +25,10 @@ public enum SchedulerEventType {
NODE_REMOVED, NODE_REMOVED,
NODE_UPDATE, NODE_UPDATE,
// Source: RMApp
APP_ADDED,
APP_REMOVED,
// Source: RMAppAttempt // Source: RMAppAttempt
APP_ATTEMPT_ADDED, APP_ATTEMPT_ADDED,
APP_ATTEMPT_REMOVED, APP_ATTEMPT_REMOVED,

View File

@ -33,7 +33,7 @@ import org.apache.hadoop.yarn.api.records.QueueACL;
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
@Private @Private
@Unstable @Unstable

View File

@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerFini
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
/** /**
@ -52,7 +52,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
*/ */
@Private @Private
@Unstable @Unstable
public class FSSchedulerApp extends SchedulerApplication { public class FSSchedulerApp extends SchedulerApplicationAttempt {
private static final Log LOG = LogFactory.getLog(FSSchedulerApp.class); private static final Log LOG = LogFactory.getLog(FSSchedulerApp.class);

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
@ -58,10 +59,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights; import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; 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.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
@ -75,8 +79,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppRepor
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
@ -151,10 +157,15 @@ public class FairScheduler implements ResourceScheduler {
// Time we last ran preemptTasksIfNecessary // Time we last ran preemptTasksIfNecessary
private long lastPreemptCheckTime; private long lastPreemptCheckTime;
// This stores per-application scheduling information, indexed by // This stores per-application scheduling information,
@VisibleForTesting
protected Map<ApplicationId, SchedulerApplication> applications =
new ConcurrentHashMap<ApplicationId, SchedulerApplication>();
// This stores per-application-attempt scheduling information, indexed by
// attempt ID's for fast lookup. // attempt ID's for fast lookup.
@VisibleForTesting @VisibleForTesting
protected Map<ApplicationAttemptId, FSSchedulerApp> applications = protected Map<ApplicationAttemptId, FSSchedulerApp> appAttempts =
new ConcurrentHashMap<ApplicationAttemptId, FSSchedulerApp>(); new ConcurrentHashMap<ApplicationAttemptId, FSSchedulerApp>();
// Nodes in the cluster, indexed by NodeId // Nodes in the cluster, indexed by NodeId
@ -253,7 +264,7 @@ public class FairScheduler implements ResourceScheduler {
private RMContainer getRMContainer(ContainerId containerId) { private RMContainer getRMContainer(ContainerId containerId) {
FSSchedulerApp application = FSSchedulerApp application =
applications.get(containerId.getApplicationAttemptId()); appAttempts.get(containerId.getApplicationAttemptId());
return (application == null) ? null : application.getRMContainer(containerId); return (application == null) ? null : application.getRMContainer(containerId);
} }
@ -591,32 +602,26 @@ public class FairScheduler implements ResourceScheduler {
* user. This will accept a new app even if the user or queue is above * user. This will accept a new app even if the user or queue is above
* configured limits, but the app will not be marked as runnable. * configured limits, but the app will not be marked as runnable.
*/ */
protected synchronized void addApplicationAttempt( protected synchronized void addApplication(ApplicationId applicationId,
ApplicationAttemptId applicationAttemptId, String queueName, String user) { String queueName, String user) {
if (queueName == null || queueName.isEmpty()) { if (queueName == null || queueName.isEmpty()) {
String message = "Reject application " + applicationAttemptId + String message = "Reject application " + applicationId +
" submitted by user " + user + " with an empty queue name."; " submitted by user " + user + " with an empty queue name.";
LOG.info(message); LOG.info(message);
rmContext.getDispatcher().getEventHandler().handle( rmContext.getDispatcher().getEventHandler()
new RMAppAttemptRejectedEvent(applicationAttemptId, message)); .handle(new RMAppRejectedEvent(applicationId, message));
return; return;
} }
RMApp rmApp = rmContext.getRMApps().get( RMApp rmApp = rmContext.getRMApps().get(applicationId);
applicationAttemptId.getApplicationId());
FSLeafQueue queue = assignToQueue(rmApp, queueName, user); FSLeafQueue queue = assignToQueue(rmApp, queueName, user);
if (queue == null) { if (queue == null) {
rmContext.getDispatcher().getEventHandler().handle( rmContext.getDispatcher().getEventHandler().handle(
new RMAppAttemptRejectedEvent(applicationAttemptId, new RMAppRejectedEvent(applicationId,
"Application rejected by queue placement policy")); "Application rejected by queue placement policy"));
return; return;
} }
FSSchedulerApp schedulerApp =
new FSSchedulerApp(applicationAttemptId, user,
queue, new ActiveUsersManager(getRootQueueMetrics()),
rmContext);
// Enforce ACLs // Enforce ACLs
UserGroupInformation userUgi = UserGroupInformation.createRemoteUser(user); UserGroupInformation userUgi = UserGroupInformation.createRemoteUser(user);
@ -625,11 +630,36 @@ public class FairScheduler implements ResourceScheduler {
String msg = "User " + userUgi.getUserName() + String msg = "User " + userUgi.getUserName() +
" cannot submit applications to queue " + queue.getName(); " cannot submit applications to queue " + queue.getName();
LOG.info(msg); LOG.info(msg);
rmContext.getDispatcher().getEventHandler().handle( rmContext.getDispatcher().getEventHandler()
new RMAppAttemptRejectedEvent(applicationAttemptId, msg)); .handle(new RMAppRejectedEvent(applicationId, msg));
return; return;
} }
SchedulerApplication application =
new SchedulerApplication(queue, user);
applications.put(applicationId, application);
LOG.info("Accepted application " + applicationId + " from user: " + user
+ ", in queue: " + queueName);
rmContext.getDispatcher().getEventHandler()
.handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
}
/**
* Add a new application attempt to the scheduler.
*/
protected synchronized void addApplicationAttempt(
ApplicationAttemptId applicationAttemptId) {
SchedulerApplication application =
applications.get(applicationAttemptId.getApplicationId());
String user = application.getUser();
FSLeafQueue queue = (FSLeafQueue) application.getQueue();
FSSchedulerApp schedulerApp =
new FSSchedulerApp(applicationAttemptId, user,
queue, new ActiveUsersManager(getRootQueueMetrics()),
rmContext);
boolean runnable = maxRunningEnforcer.canAppBeRunnable(queue, user); boolean runnable = maxRunningEnforcer.canAppBeRunnable(queue, user);
queue.addApp(schedulerApp, runnable); queue.addApp(schedulerApp, runnable);
if (runnable) { if (runnable) {
@ -639,16 +669,14 @@ public class FairScheduler implements ResourceScheduler {
} }
queue.getMetrics().submitApp(user, applicationAttemptId.getAttemptId()); queue.getMetrics().submitApp(user, applicationAttemptId.getAttemptId());
appAttempts.put(applicationAttemptId, schedulerApp);
applications.put(applicationAttemptId, schedulerApp); LOG.info("Added Application Attempt " + applicationAttemptId
+ " to scheduler from user: " + user + ", currently active: "
LOG.info("Application Submission: " + applicationAttemptId + + appAttempts.size());
", user: "+ user +
", currently active: " + applications.size());
rmContext.getDispatcher().getEventHandler().handle( rmContext.getDispatcher().getEventHandler().handle(
new RMAppAttemptEvent(applicationAttemptId, new RMAppAttemptEvent(applicationAttemptId,
RMAppAttemptEventType.APP_ACCEPTED)); RMAppAttemptEventType.ATTEMPT_ADDED));
} }
@VisibleForTesting @VisibleForTesting
@ -674,13 +702,18 @@ public class FairScheduler implements ResourceScheduler {
return queue; return queue;
} }
private synchronized void removeApplication(ApplicationId applicationId,
RMAppState finalState) {
applications.remove(applicationId);
}
private synchronized void removeApplicationAttempt( private synchronized void removeApplicationAttempt(
ApplicationAttemptId applicationAttemptId, ApplicationAttemptId applicationAttemptId,
RMAppAttemptState rmAppAttemptFinalState) { RMAppAttemptState rmAppAttemptFinalState) {
LOG.info("Application " + applicationAttemptId + " is done." + LOG.info("Application " + applicationAttemptId + " is done." +
" finalState=" + rmAppAttemptFinalState); " finalState=" + rmAppAttemptFinalState);
FSSchedulerApp application = applications.get(applicationAttemptId); FSSchedulerApp application = appAttempts.get(applicationAttemptId);
if (application == null) { if (application == null) {
LOG.info("Unknown application " + applicationAttemptId + " has completed!"); LOG.info("Unknown application " + applicationAttemptId + " has completed!");
@ -720,7 +753,7 @@ public class FairScheduler implements ResourceScheduler {
} }
// Remove from our data-structure // Remove from our data-structure
applications.remove(applicationAttemptId); appAttempts.remove(applicationAttemptId);
} }
/** /**
@ -737,7 +770,7 @@ public class FairScheduler implements ResourceScheduler {
// Get the application for the finished container // Get the application for the finished container
ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId(); ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId();
FSSchedulerApp application = applications.get(applicationAttemptId); FSSchedulerApp application = appAttempts.get(applicationAttemptId);
if (application == null) { if (application == null) {
LOG.info("Container " + container + " of" + LOG.info("Container " + container + " of" +
" unknown application " + applicationAttemptId + " unknown application " + applicationAttemptId +
@ -811,7 +844,7 @@ public class FairScheduler implements ResourceScheduler {
List<ResourceRequest> ask, List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) { List<ResourceRequest> ask, List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) {
// Make sure this application exists // Make sure this application exists
FSSchedulerApp application = applications.get(appAttemptId); FSSchedulerApp application = appAttempts.get(appAttemptId);
if (application == null) { if (application == null) {
LOG.info("Calling allocate on removed " + LOG.info("Calling allocate on removed " +
"or non existant application " + appAttemptId); "or non existant application " + appAttemptId);
@ -882,7 +915,7 @@ public class FairScheduler implements ResourceScheduler {
private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode node) { private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode node) {
// Get the application for the finished container // Get the application for the finished container
ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId();
FSSchedulerApp application = applications.get(applicationAttemptId); FSSchedulerApp application = appAttempts.get(applicationAttemptId);
if (application == null) { if (application == null) {
LOG.info("Unknown application: " + applicationAttemptId + LOG.info("Unknown application: " + applicationAttemptId +
" launched container " + containerId + " launched container " + containerId +
@ -1025,23 +1058,23 @@ public class FairScheduler implements ResourceScheduler {
} }
public FSSchedulerApp getSchedulerApp(ApplicationAttemptId appAttemptId) { public FSSchedulerApp getSchedulerApp(ApplicationAttemptId appAttemptId) {
return applications.get(appAttemptId); return appAttempts.get(appAttemptId);
} }
@Override @Override
public SchedulerAppReport getSchedulerAppInfo( public SchedulerAppReport getSchedulerAppInfo(
ApplicationAttemptId appAttemptId) { ApplicationAttemptId appAttemptId) {
if (!applications.containsKey(appAttemptId)) { if (!appAttempts.containsKey(appAttemptId)) {
LOG.error("Request for appInfo of unknown attempt" + appAttemptId); LOG.error("Request for appInfo of unknown attempt" + appAttemptId);
return null; return null;
} }
return new SchedulerAppReport(applications.get(appAttemptId)); return new SchedulerAppReport(appAttempts.get(appAttemptId));
} }
@Override @Override
public ApplicationResourceUsageReport getAppResourceUsageReport( public ApplicationResourceUsageReport getAppResourceUsageReport(
ApplicationAttemptId appAttemptId) { ApplicationAttemptId appAttemptId) {
FSSchedulerApp app = applications.get(appAttemptId); FSSchedulerApp app = appAttempts.get(appAttemptId);
if (app == null) { if (app == null) {
LOG.error("Request for appInfo of unknown attempt" + appAttemptId); LOG.error("Request for appInfo of unknown attempt" + appAttemptId);
return null; return null;
@ -1090,15 +1123,29 @@ public class FairScheduler implements ResourceScheduler {
NodeUpdateSchedulerEvent nodeUpdatedEvent = (NodeUpdateSchedulerEvent)event; NodeUpdateSchedulerEvent nodeUpdatedEvent = (NodeUpdateSchedulerEvent)event;
nodeUpdate(nodeUpdatedEvent.getRMNode()); nodeUpdate(nodeUpdatedEvent.getRMNode());
break; break;
case APP_ADDED:
if (!(event instanceof AppAddedSchedulerEvent)) {
throw new RuntimeException("Unexpected event type: " + event);
}
AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
addApplication(appAddedEvent.getApplicationId(),
appAddedEvent.getQueue(), appAddedEvent.getUser());
break;
case APP_REMOVED:
if (!(event instanceof AppRemovedSchedulerEvent)) {
throw new RuntimeException("Unexpected event type: " + event);
}
AppRemovedSchedulerEvent appRemovedEvent = (AppRemovedSchedulerEvent)event;
removeApplication(appRemovedEvent.getApplicationID(),
appRemovedEvent.getFinalState());
break;
case APP_ATTEMPT_ADDED: case APP_ATTEMPT_ADDED:
if (!(event instanceof AppAttemptAddedSchedulerEvent)) { if (!(event instanceof AppAttemptAddedSchedulerEvent)) {
throw new RuntimeException("Unexpected event type: " + event); throw new RuntimeException("Unexpected event type: " + event);
} }
AppAttemptAddedSchedulerEvent appAttemptAddedEvent = AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
(AppAttemptAddedSchedulerEvent) event; (AppAttemptAddedSchedulerEvent) event;
String queue = appAttemptAddedEvent.getQueue(); addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId());
addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(),
queue, appAttemptAddedEvent.getUser());
break; break;
case APP_ATTEMPT_REMOVED: case APP_ATTEMPT_REMOVED:
if (!(event instanceof AppAttemptRemovedSchedulerEvent)) { if (!(event instanceof AppAttemptRemovedSchedulerEvent)) {

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
@ -58,6 +59,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@ -74,12 +78,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
@ -116,9 +123,13 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
private Resource maximumAllocation; private Resource maximumAllocation;
private boolean usePortForNodeName; private boolean usePortForNodeName;
@VisibleForTesting
protected Map<ApplicationId, SchedulerApplication> applications =
new ConcurrentSkipListMap<ApplicationId, SchedulerApplication>();
// Use ConcurrentSkipListMap because applications need to be ordered // Use ConcurrentSkipListMap because applications need to be ordered
@VisibleForTesting @VisibleForTesting
protected Map<ApplicationAttemptId, FiCaSchedulerApp> applications protected Map<ApplicationAttemptId, FiCaSchedulerApp> appAttempts
= new ConcurrentSkipListMap<ApplicationAttemptId, FiCaSchedulerApp>(); = new ConcurrentSkipListMap<ApplicationAttemptId, FiCaSchedulerApp>();
private ActiveUsersManager activeUsersManager; private ActiveUsersManager activeUsersManager;
@ -327,7 +338,7 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
@VisibleForTesting @VisibleForTesting
FiCaSchedulerApp getApplication( FiCaSchedulerApp getApplication(
ApplicationAttemptId applicationAttemptId) { ApplicationAttemptId applicationAttemptId) {
return applications.get(applicationAttemptId); return appAttempts.get(applicationAttemptId);
} }
@Override @Override
@ -348,19 +359,43 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
return nodes.get(nodeId); return nodes.get(nodeId);
} }
private synchronized void addApplicationAttempt(ApplicationAttemptId appAttemptId, private synchronized void addApplication(ApplicationId applicationId,
String user) { String queue, String user) {
SchedulerApplication application =
new SchedulerApplication(null, user);
applications.put(applicationId, application);
LOG.info("Accepted application " + applicationId + " from user: " + user);
rmContext.getDispatcher().getEventHandler()
.handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
}
private synchronized void addApplicationAttempt(
ApplicationAttemptId appAttemptId) {
SchedulerApplication application =
applications.get(appAttemptId.getApplicationId());
String user = application.getUser();
// TODO: Fix store // TODO: Fix store
FiCaSchedulerApp schedulerApp = FiCaSchedulerApp schedulerApp =
new FiCaSchedulerApp(appAttemptId, user, DEFAULT_QUEUE, activeUsersManager, new FiCaSchedulerApp(appAttemptId, user, DEFAULT_QUEUE,
this.rmContext); activeUsersManager, this.rmContext);
applications.put(appAttemptId, schedulerApp); appAttempts.put(appAttemptId, schedulerApp);
metrics.submitApp(user, appAttemptId.getAttemptId()); metrics.submitApp(user, appAttemptId.getAttemptId());
LOG.info("Application Submission: " + appAttemptId.getApplicationId() + LOG.info("Added Application Attempt " + appAttemptId
" from " + user + ", currently active: " + applications.size()); + " to scheduler from user " + application.getUser()
+ ", currently active: " + appAttempts.size());
rmContext.getDispatcher().getEventHandler().handle( rmContext.getDispatcher().getEventHandler().handle(
new RMAppAttemptEvent(appAttemptId, new RMAppAttemptEvent(appAttemptId,
RMAppAttemptEventType.APP_ACCEPTED)); RMAppAttemptEventType.ATTEMPT_ADDED));
}
private synchronized void doneApplication(ApplicationId applicationId,
RMAppState finalState) {
SchedulerApplication application = applications.get(applicationId);
// Inform the activeUsersManager
activeUsersManager.deactivateApplication(application.getUser(),
applicationId);
applications.remove(applicationId);
} }
private synchronized void doneApplicationAttempt( private synchronized void doneApplicationAttempt(
@ -382,17 +417,11 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
RMContainerEventType.KILL); RMContainerEventType.KILL);
} }
// Inform the activeUsersManager
synchronized (application) {
activeUsersManager.deactivateApplication(
application.getUser(), application.getApplicationId());
}
// Clean up pending requests, metrics etc. // Clean up pending requests, metrics etc.
application.stop(rmAppAttemptFinalState); application.stop(rmAppAttemptFinalState);
// Remove the application // Remove the application
applications.remove(applicationAttemptId); appAttempts.remove(applicationAttemptId);
} }
/** /**
@ -403,10 +432,10 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
private void assignContainers(FiCaSchedulerNode node) { private void assignContainers(FiCaSchedulerNode node) {
LOG.debug("assignContainers:" + LOG.debug("assignContainers:" +
" node=" + node.getRMNode().getNodeAddress() + " node=" + node.getRMNode().getNodeAddress() +
" #applications=" + applications.size()); " #applications=" + appAttempts.size());
// Try to assign containers to applications in fifo order // Try to assign containers to applications in fifo order
for (Map.Entry<ApplicationAttemptId, FiCaSchedulerApp> e : applications for (Map.Entry<ApplicationAttemptId, FiCaSchedulerApp> e : appAttempts
.entrySet()) { .entrySet()) {
FiCaSchedulerApp application = e.getValue(); FiCaSchedulerApp application = e.getValue();
LOG.debug("pre-assignContainers"); LOG.debug("pre-assignContainers");
@ -445,7 +474,7 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
// Update the applications' headroom to correctly take into // Update the applications' headroom to correctly take into
// account the containers assigned in this update. // account the containers assigned in this update.
for (FiCaSchedulerApp application : applications.values()) { for (FiCaSchedulerApp application : appAttempts.values()) {
application.setHeadroom(Resources.subtract(clusterResource, usedResource)); application.setHeadroom(Resources.subtract(clusterResource, usedResource));
} }
} }
@ -697,12 +726,25 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
nodeUpdate(nodeUpdatedEvent.getRMNode()); nodeUpdate(nodeUpdatedEvent.getRMNode());
} }
break; break;
case APP_ADDED:
{
AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
addApplication(appAddedEvent.getApplicationId(),
appAddedEvent.getQueue(), appAddedEvent.getUser());
}
break;
case APP_REMOVED:
{
AppRemovedSchedulerEvent appRemovedEvent = (AppRemovedSchedulerEvent)event;
doneApplication(appRemovedEvent.getApplicationID(),
appRemovedEvent.getFinalState());
}
break;
case APP_ATTEMPT_ADDED: case APP_ATTEMPT_ADDED:
{ {
AppAttemptAddedSchedulerEvent appAttemptAddedEvent = AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
(AppAttemptAddedSchedulerEvent) event; (AppAttemptAddedSchedulerEvent) event;
addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(), addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId());
appAttemptAddedEvent.getUser());
} }
break; break;
case APP_ATTEMPT_REMOVED: case APP_ATTEMPT_REMOVED:
@ -867,8 +909,8 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
public synchronized List<ApplicationAttemptId> getAppsInQueue(String queueName) { public synchronized List<ApplicationAttemptId> getAppsInQueue(String queueName) {
if (queueName.equals(DEFAULT_QUEUE.getQueueName())) { if (queueName.equals(DEFAULT_QUEUE.getQueueName())) {
List<ApplicationAttemptId> apps = new ArrayList<ApplicationAttemptId>( List<ApplicationAttemptId> apps = new ArrayList<ApplicationAttemptId>(
applications.size()); appAttempts.size());
for (FiCaSchedulerApp app : applications.values()) { for (FiCaSchedulerApp app : appAttempts.values()) {
apps.add(app.getApplicationAttemptId()); apps.add(app.getApplicationAttemptId());
} }
return apps; return apps;

View File

@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.Task.State;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
@ -166,9 +167,12 @@ public class Application {
resourceManager.getClientRMService().submitApplication(request); resourceManager.getClientRMService().submitApplication(request);
// Notify scheduler // Notify scheduler
AppAttemptAddedSchedulerEvent appAddedEvent1 = new AppAttemptAddedSchedulerEvent( AppAddedSchedulerEvent addAppEvent =
this.applicationAttemptId, this.queue, this.user); new AppAddedSchedulerEvent(this.applicationId, this.queue, "user");
scheduler.handle(appAddedEvent1); scheduler.handle(addAppEvent);
AppAttemptAddedSchedulerEvent addAttemptEvent =
new AppAttemptAddedSchedulerEvent(this.applicationAttemptId);
scheduler.handle(addAttemptEvent);
} }
public synchronized void addResourceRequestSpec( public synchronized void addResourceRequestSpec(

View File

@ -649,7 +649,7 @@ public class TestClientRMService {
.currentTimeMillis(), "YARN")); .currentTimeMillis(), "YARN"));
ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(applicationId3, 1); ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(applicationId3, 1);
RMAppAttemptImpl rmAppAttemptImpl = new RMAppAttemptImpl(attemptId, RMAppAttemptImpl rmAppAttemptImpl = new RMAppAttemptImpl(attemptId,
rmContext, yarnScheduler, null, asContext, config, null); rmContext, yarnScheduler, null, asContext, config);
when(app.getCurrentAppAttempt()).thenReturn(rmAppAttemptImpl); when(app.getCurrentAppAttempt()).thenReturn(rmAppAttemptImpl);
return app; return app;
} }

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
@ -297,9 +298,12 @@ public class TestFifoScheduler {
ApplicationId appId1 = BuilderUtils.newApplicationId(100, 1); ApplicationId appId1 = BuilderUtils.newApplicationId(100, 1);
ApplicationAttemptId appAttemptId1 = BuilderUtils.newApplicationAttemptId( ApplicationAttemptId appAttemptId1 = BuilderUtils.newApplicationAttemptId(
appId1, 1); appId1, 1);
SchedulerEvent event1 = SchedulerEvent appEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId1, "queue", "user"); new AppAddedSchedulerEvent(appId1, "queue", "user");
fs.handle(event1); fs.handle(appEvent);
SchedulerEvent attemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId1);
fs.handle(attemptEvent);
List<ContainerId> emptyId = new ArrayList<ContainerId>(); List<ContainerId> emptyId = new ArrayList<ContainerId>();
List<ResourceRequest> emptyAsk = new ArrayList<ResourceRequest>(); List<ResourceRequest> emptyAsk = new ArrayList<ResourceRequest>();
@ -388,16 +392,22 @@ public class TestFifoScheduler {
ApplicationId appId1 = BuilderUtils.newApplicationId(100, 1); ApplicationId appId1 = BuilderUtils.newApplicationId(100, 1);
ApplicationAttemptId appAttemptId1 = BuilderUtils.newApplicationAttemptId( ApplicationAttemptId appAttemptId1 = BuilderUtils.newApplicationAttemptId(
appId1, 1); appId1, 1);
SchedulerEvent event1 = SchedulerEvent appEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId1, "queue", "user"); new AppAddedSchedulerEvent(appId1, "queue", "user");
fs.handle(event1); fs.handle(appEvent);
SchedulerEvent attemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId1);
fs.handle(attemptEvent);
ApplicationId appId2 = BuilderUtils.newApplicationId(200, 2); ApplicationId appId2 = BuilderUtils.newApplicationId(200, 2);
ApplicationAttemptId appAttemptId2 = BuilderUtils.newApplicationAttemptId( ApplicationAttemptId appAttemptId2 = BuilderUtils.newApplicationAttemptId(
appId2, 1); appId2, 1);
SchedulerEvent event2 = SchedulerEvent appEvent2 =
new AppAttemptAddedSchedulerEvent(appAttemptId2, "queue", "user"); new AppAddedSchedulerEvent(appId2, "queue", "user");
fs.handle(event2); fs.handle(appEvent2);
SchedulerEvent attemptEvent2 =
new AppAttemptAddedSchedulerEvent(appAttemptId2);
fs.handle(attemptEvent2);
List<ContainerId> emptyId = new ArrayList<ContainerId>(); List<ContainerId> emptyId = new ArrayList<ContainerId>();
List<ResourceRequest> emptyAsk = new ArrayList<ResourceRequest>(); List<ResourceRequest> emptyAsk = new ArrayList<ResourceRequest>();

View File

@ -248,7 +248,7 @@ public class TestRMRestart {
// verify correct number of attempts and other data // verify correct number of attempts and other data
RMApp loadedApp1 = rm2.getRMContext().getRMApps().get(app1.getApplicationId()); RMApp loadedApp1 = rm2.getRMContext().getRMApps().get(app1.getApplicationId());
Assert.assertNotNull(loadedApp1); Assert.assertNotNull(loadedApp1);
//Assert.assertEquals(1, loadedApp1.getAppAttempts().size()); Assert.assertEquals(1, loadedApp1.getAppAttempts().size());
Assert.assertEquals(app1.getApplicationSubmissionContext() Assert.assertEquals(app1.getApplicationSubmissionContext()
.getApplicationId(), loadedApp1.getApplicationSubmissionContext() .getApplicationId(), loadedApp1.getApplicationSubmissionContext()
.getApplicationId()); .getApplicationId());
@ -261,7 +261,7 @@ public class TestRMRestart {
.getApplicationId()); .getApplicationId());
// verify state machine kicked into expected states // verify state machine kicked into expected states
rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.RUNNING); rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.ACCEPTED);
rm2.waitForState(loadedApp2.getApplicationId(), RMAppState.ACCEPTED); rm2.waitForState(loadedApp2.getApplicationId(), RMAppState.ACCEPTED);
// verify attempts for apps // verify attempts for apps
@ -299,7 +299,11 @@ public class TestRMRestart {
nm2.registerNode(); nm2.registerNode();
rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.ACCEPTED); rm2.waitForState(loadedApp1.getApplicationId(), RMAppState.ACCEPTED);
Assert.assertEquals(2, loadedApp1.getAppAttempts().size()); // wait for the 2nd attempt to be started.
int timeoutSecs = 0;
while (loadedApp1.getAppAttempts().size() != 2 && timeoutSecs++ < 40) {;
Thread.sleep(200);
}
// verify no more reboot response sent // verify no more reboot response sent
hbResponse = nm1.nodeHeartbeat(true); hbResponse = nm1.nodeHeartbeat(true);
@ -476,10 +480,10 @@ public class TestRMRestart {
Assert.assertEquals(NodeAction.RESYNC, res.getNodeAction()); Assert.assertEquals(NodeAction.RESYNC, res.getNodeAction());
RMApp rmApp = rm2.getRMContext().getRMApps().get(app1.getApplicationId()); RMApp rmApp = rm2.getRMContext().getRMApps().get(app1.getApplicationId());
// application should be in running state // application should be in ACCEPTED state
rm2.waitForState(app1.getApplicationId(), RMAppState.RUNNING); rm2.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
Assert.assertEquals(RMAppState.RUNNING, rmApp.getState()); Assert.assertEquals(RMAppState.ACCEPTED, rmApp.getState());
// new attempt should not be started // new attempt should not be started
Assert.assertEquals(2, rmApp.getAppAttempts().size()); Assert.assertEquals(2, rmApp.getAppAttempts().size());
// am1 attempt should be in FAILED state where as am2 attempt should be in // am1 attempt should be in FAILED state where as am2 attempt should be in
@ -516,9 +520,9 @@ public class TestRMRestart {
nm1.setResourceTrackerService(rm3.getResourceTrackerService()); nm1.setResourceTrackerService(rm3.getResourceTrackerService());
rmApp = rm3.getRMContext().getRMApps().get(app1.getApplicationId()); rmApp = rm3.getRMContext().getRMApps().get(app1.getApplicationId());
// application should be in running state // application should be in ACCEPTED state
rm3.waitForState(app1.getApplicationId(), RMAppState.RUNNING); rm3.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
Assert.assertEquals(rmApp.getState(), RMAppState.RUNNING); Assert.assertEquals(rmApp.getState(), RMAppState.ACCEPTED);
// new attempt should not be started // new attempt should not be started
Assert.assertEquals(3, rmApp.getAppAttempts().size()); Assert.assertEquals(3, rmApp.getAppAttempts().size());
// am1 and am2 attempts should be in FAILED state where as am3 should be // am1 and am2 attempts should be in FAILED state where as am3 should be
@ -562,6 +566,11 @@ public class TestRMRestart {
rmApp = rm4.getRMContext().getRMApps().get(app1.getApplicationId()); rmApp = rm4.getRMContext().getRMApps().get(app1.getApplicationId());
rm4.waitForState(rmApp.getApplicationId(), RMAppState.ACCEPTED); rm4.waitForState(rmApp.getApplicationId(), RMAppState.ACCEPTED);
// wait for the attempt to be created.
int timeoutSecs = 0;
while (rmApp.getAppAttempts().size() != 2 && timeoutSecs++ < 40) {
Thread.sleep(200);
}
Assert.assertEquals(4, rmApp.getAppAttempts().size()); Assert.assertEquals(4, rmApp.getAppAttempts().size());
Assert.assertEquals(RMAppState.ACCEPTED, rmApp.getState()); Assert.assertEquals(RMAppState.ACCEPTED, rmApp.getState());
rm4.waitForState(latestAppAttemptId, RMAppAttemptState.SCHEDULED); rm4.waitForState(latestAppAttemptId, RMAppAttemptState.SCHEDULED);

View File

@ -567,7 +567,9 @@ public class TestRMAppTransitions {
RMAppEventType.KILL); RMAppEventType.KILL);
application.handle(event); application.handle(event);
rmDispatcher.await(); rmDispatcher.await();
assertAppAndAttemptKilled(application); sendAppUpdateSavedEvent(application);
assertKilled(application);
assertAppFinalStateSaved(application);
} }
@Test @Test
@ -582,7 +584,7 @@ public class TestRMAppTransitions {
new RMAppFailedAttemptEvent(application.getApplicationId(), new RMAppFailedAttemptEvent(application.getApplicationId(),
RMAppEventType.ATTEMPT_FAILED, ""); RMAppEventType.ATTEMPT_FAILED, "");
application.handle(event); application.handle(event);
assertAppState(RMAppState.SUBMITTED, application); assertAppState(RMAppState.ACCEPTED, application);
event = event =
new RMAppEvent(application.getApplicationId(), new RMAppEvent(application.getApplicationId(),
RMAppEventType.APP_ACCEPTED); RMAppEventType.APP_ACCEPTED);
@ -612,7 +614,9 @@ public class TestRMAppTransitions {
RMAppEventType.KILL); RMAppEventType.KILL);
application.handle(event); application.handle(event);
rmDispatcher.await(); rmDispatcher.await();
assertAppAndAttemptKilled(application); sendAppUpdateSavedEvent(application);
assertKilled(application);
assertAppFinalStateSaved(application);
} }
@Test @Test
@ -654,7 +658,7 @@ public class TestRMAppTransitions {
RMAppEventType.ATTEMPT_FAILED, ""); RMAppEventType.ATTEMPT_FAILED, "");
application.handle(event); application.handle(event);
rmDispatcher.await(); rmDispatcher.await();
assertAppState(RMAppState.SUBMITTED, application); assertAppState(RMAppState.ACCEPTED, application);
appAttempt = application.getCurrentAppAttempt(); appAttempt = application.getCurrentAppAttempt();
Assert.assertEquals(++expectedAttemptId, Assert.assertEquals(++expectedAttemptId,
appAttempt.getAppAttemptId().getAttemptId()); appAttempt.getAppAttemptId().getAttemptId());

View File

@ -79,7 +79,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
@ -258,7 +257,7 @@ public class TestRMAppAttemptTransitions {
application = mock(RMApp.class); application = mock(RMApp.class);
applicationAttempt = applicationAttempt =
new RMAppAttemptImpl(applicationAttemptId, rmContext, scheduler, new RMAppAttemptImpl(applicationAttemptId, rmContext, scheduler,
masterService, submissionContext, new Configuration(), user); masterService, submissionContext, new Configuration());
when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt); when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt);
when(application.getApplicationId()).thenReturn(applicationId); when(application.getApplicationId()).thenReturn(applicationId);
@ -408,9 +407,6 @@ public class TestRMAppAttemptTransitions {
assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001); assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001);
assertEquals(0, applicationAttempt.getRanNodes().size()); assertEquals(0, applicationAttempt.getRanNodes().size());
assertNull(applicationAttempt.getFinalApplicationStatus()); assertNull(applicationAttempt.getFinalApplicationStatus());
// Check events
verify(application).handle(any(RMAppEvent.class));
} }
/** /**
@ -446,7 +442,7 @@ public class TestRMAppAttemptTransitions {
assertEquals(0, applicationAttempt.getRanNodes().size()); assertEquals(0, applicationAttempt.getRanNodes().size());
// Check events // Check events
verify(application, times(2)).handle(any(RMAppFailedAttemptEvent.class)); verify(application, times(1)).handle(any(RMAppFailedAttemptEvent.class));
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1); verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
verifyAttemptFinalStateSaved(); verifyAttemptFinalStateSaved();
} }
@ -544,7 +540,7 @@ public class TestRMAppAttemptTransitions {
applicationAttempt.handle( applicationAttempt.handle(
new RMAppAttemptEvent( new RMAppAttemptEvent(
applicationAttempt.getAppAttemptId(), applicationAttempt.getAppAttemptId(),
RMAppAttemptEventType.APP_ACCEPTED)); RMAppAttemptEventType.ATTEMPT_ADDED));
if(unmanagedAM){ if(unmanagedAM){
assertEquals(RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING, assertEquals(RMAppAttemptState.LAUNCHED_UNMANAGED_SAVING,
@ -704,16 +700,6 @@ public class TestRMAppAttemptTransitions {
testAppAttemptRecoveredState(); testAppAttemptRecoveredState();
} }
@Test
public void testSubmittedToFailed() {
submitApplicationAttempt();
String message = "Rejected";
applicationAttempt.handle(
new RMAppAttemptRejectedEvent(
applicationAttempt.getAppAttemptId(), message));
testAppAttemptSubmittedToFailedState(message);
}
@Test @Test
public void testSubmittedToKilled() { public void testSubmittedToKilled() {
submitApplicationAttempt(); submitApplicationAttempt();

View File

@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl; import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException; import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException;
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException; import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.ipc.YarnRPC;
@ -58,8 +59,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMWithAMS; import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMWithAMS;
import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager; import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
@ -378,4 +383,24 @@ public class TestSchedulerUtils {
ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x"); ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x");
Assert.assertEquals(ContainerExitStatus.PREEMPTED, cd.getExitStatus()); Assert.assertEquals(ContainerExitStatus.PREEMPTED, cd.getExitStatus());
} }
public static <T> SchedulerApplication verifyAppAddedAndRemovedFromScheduler(
final Map<ApplicationId, SchedulerApplication> applications,
EventHandler<SchedulerEvent> handler, String queueName) throws Exception {
ApplicationId appId =
ApplicationId.newInstance(System.currentTimeMillis(), 1);
AppAddedSchedulerEvent appAddedEvent =
new AppAddedSchedulerEvent(appId, queueName, "user");
handler.handle(appAddedEvent);
SchedulerApplication app = applications.get(appId);
// verify application is added.
Assert.assertNotNull(app);
Assert.assertEquals("user", app.getUser());
AppRemovedSchedulerEvent appRemoveEvent =
new AppRemovedSchedulerEvent(appId, RMAppState.FINISHED);
handler.handle(appRemoveEvent);
Assert.assertNull(applications.get(appId));
return app;
}
} }

View File

@ -304,7 +304,7 @@ public class TestApplicationLimits {
int APPLICATION_ID = 0; int APPLICATION_ID = 0;
// Submit first application // Submit first application
FiCaSchedulerApp app_0 = getMockApplication(APPLICATION_ID++, user_0); FiCaSchedulerApp app_0 = getMockApplication(APPLICATION_ID++, user_0);
queue.submitApplication(app_0, user_0, A); queue.submitApplicationAttempt(app_0, user_0);
assertEquals(1, queue.getNumActiveApplications()); assertEquals(1, queue.getNumActiveApplications());
assertEquals(0, queue.getNumPendingApplications()); assertEquals(0, queue.getNumPendingApplications());
assertEquals(1, queue.getNumActiveApplications(user_0)); assertEquals(1, queue.getNumActiveApplications(user_0));
@ -312,7 +312,7 @@ public class TestApplicationLimits {
// Submit second application // Submit second application
FiCaSchedulerApp app_1 = getMockApplication(APPLICATION_ID++, user_0); FiCaSchedulerApp app_1 = getMockApplication(APPLICATION_ID++, user_0);
queue.submitApplication(app_1, user_0, A); queue.submitApplicationAttempt(app_1, user_0);
assertEquals(2, queue.getNumActiveApplications()); assertEquals(2, queue.getNumActiveApplications());
assertEquals(0, queue.getNumPendingApplications()); assertEquals(0, queue.getNumPendingApplications());
assertEquals(2, queue.getNumActiveApplications(user_0)); assertEquals(2, queue.getNumActiveApplications(user_0));
@ -320,14 +320,14 @@ public class TestApplicationLimits {
// Submit third application, should remain pending // Submit third application, should remain pending
FiCaSchedulerApp app_2 = getMockApplication(APPLICATION_ID++, user_0); FiCaSchedulerApp app_2 = getMockApplication(APPLICATION_ID++, user_0);
queue.submitApplication(app_2, user_0, A); queue.submitApplicationAttempt(app_2, user_0);
assertEquals(2, queue.getNumActiveApplications()); assertEquals(2, queue.getNumActiveApplications());
assertEquals(1, queue.getNumPendingApplications()); assertEquals(1, queue.getNumPendingApplications());
assertEquals(2, queue.getNumActiveApplications(user_0)); assertEquals(2, queue.getNumActiveApplications(user_0));
assertEquals(1, queue.getNumPendingApplications(user_0)); assertEquals(1, queue.getNumPendingApplications(user_0));
// Finish one application, app_2 should be activated // Finish one application, app_2 should be activated
queue.finishApplication(app_0, A); queue.finishApplicationAttempt(app_0, A);
assertEquals(2, queue.getNumActiveApplications()); assertEquals(2, queue.getNumActiveApplications());
assertEquals(0, queue.getNumPendingApplications()); assertEquals(0, queue.getNumPendingApplications());
assertEquals(2, queue.getNumActiveApplications(user_0)); assertEquals(2, queue.getNumActiveApplications(user_0));
@ -335,7 +335,7 @@ public class TestApplicationLimits {
// Submit another one for user_0 // Submit another one for user_0
FiCaSchedulerApp app_3 = getMockApplication(APPLICATION_ID++, user_0); FiCaSchedulerApp app_3 = getMockApplication(APPLICATION_ID++, user_0);
queue.submitApplication(app_3, user_0, A); queue.submitApplicationAttempt(app_3, user_0);
assertEquals(2, queue.getNumActiveApplications()); assertEquals(2, queue.getNumActiveApplications());
assertEquals(1, queue.getNumPendingApplications()); assertEquals(1, queue.getNumPendingApplications());
assertEquals(2, queue.getNumActiveApplications(user_0)); assertEquals(2, queue.getNumActiveApplications(user_0));
@ -346,7 +346,7 @@ public class TestApplicationLimits {
// Submit first app for user_1 // Submit first app for user_1
FiCaSchedulerApp app_4 = getMockApplication(APPLICATION_ID++, user_1); FiCaSchedulerApp app_4 = getMockApplication(APPLICATION_ID++, user_1);
queue.submitApplication(app_4, user_1, A); queue.submitApplicationAttempt(app_4, user_1);
assertEquals(3, queue.getNumActiveApplications()); assertEquals(3, queue.getNumActiveApplications());
assertEquals(1, queue.getNumPendingApplications()); assertEquals(1, queue.getNumPendingApplications());
assertEquals(2, queue.getNumActiveApplications(user_0)); assertEquals(2, queue.getNumActiveApplications(user_0));
@ -356,7 +356,7 @@ public class TestApplicationLimits {
// Submit second app for user_1, should block due to queue-limit // Submit second app for user_1, should block due to queue-limit
FiCaSchedulerApp app_5 = getMockApplication(APPLICATION_ID++, user_1); FiCaSchedulerApp app_5 = getMockApplication(APPLICATION_ID++, user_1);
queue.submitApplication(app_5, user_1, A); queue.submitApplicationAttempt(app_5, user_1);
assertEquals(3, queue.getNumActiveApplications()); assertEquals(3, queue.getNumActiveApplications());
assertEquals(2, queue.getNumPendingApplications()); assertEquals(2, queue.getNumPendingApplications());
assertEquals(2, queue.getNumActiveApplications(user_0)); assertEquals(2, queue.getNumActiveApplications(user_0));
@ -365,7 +365,7 @@ public class TestApplicationLimits {
assertEquals(1, queue.getNumPendingApplications(user_1)); assertEquals(1, queue.getNumPendingApplications(user_1));
// Now finish one app of user_1 so app_5 should be activated // Now finish one app of user_1 so app_5 should be activated
queue.finishApplication(app_4, A); queue.finishApplicationAttempt(app_4, A);
assertEquals(3, queue.getNumActiveApplications()); assertEquals(3, queue.getNumActiveApplications());
assertEquals(1, queue.getNumPendingApplications()); assertEquals(1, queue.getNumPendingApplications());
assertEquals(2, queue.getNumActiveApplications(user_0)); assertEquals(2, queue.getNumActiveApplications(user_0));
@ -385,7 +385,7 @@ public class TestApplicationLimits {
// Submit first application // Submit first application
FiCaSchedulerApp app_0 = getMockApplication(APPLICATION_ID++, user_0); FiCaSchedulerApp app_0 = getMockApplication(APPLICATION_ID++, user_0);
queue.submitApplication(app_0, user_0, A); queue.submitApplicationAttempt(app_0, user_0);
assertEquals(1, queue.getNumActiveApplications()); assertEquals(1, queue.getNumActiveApplications());
assertEquals(0, queue.getNumPendingApplications()); assertEquals(0, queue.getNumPendingApplications());
assertEquals(1, queue.getNumActiveApplications(user_0)); assertEquals(1, queue.getNumActiveApplications(user_0));
@ -394,7 +394,7 @@ public class TestApplicationLimits {
// Submit second application // Submit second application
FiCaSchedulerApp app_1 = getMockApplication(APPLICATION_ID++, user_0); FiCaSchedulerApp app_1 = getMockApplication(APPLICATION_ID++, user_0);
queue.submitApplication(app_1, user_0, A); queue.submitApplicationAttempt(app_1, user_0);
assertEquals(2, queue.getNumActiveApplications()); assertEquals(2, queue.getNumActiveApplications());
assertEquals(0, queue.getNumPendingApplications()); assertEquals(0, queue.getNumPendingApplications());
assertEquals(2, queue.getNumActiveApplications(user_0)); assertEquals(2, queue.getNumActiveApplications(user_0));
@ -403,7 +403,7 @@ public class TestApplicationLimits {
// Submit third application, should remain pending // Submit third application, should remain pending
FiCaSchedulerApp app_2 = getMockApplication(APPLICATION_ID++, user_0); FiCaSchedulerApp app_2 = getMockApplication(APPLICATION_ID++, user_0);
queue.submitApplication(app_2, user_0, A); queue.submitApplicationAttempt(app_2, user_0);
assertEquals(2, queue.getNumActiveApplications()); assertEquals(2, queue.getNumActiveApplications());
assertEquals(1, queue.getNumPendingApplications()); assertEquals(1, queue.getNumPendingApplications());
assertEquals(2, queue.getNumActiveApplications(user_0)); assertEquals(2, queue.getNumActiveApplications(user_0));
@ -412,7 +412,7 @@ public class TestApplicationLimits {
// Submit fourth application, should remain pending // Submit fourth application, should remain pending
FiCaSchedulerApp app_3 = getMockApplication(APPLICATION_ID++, user_0); FiCaSchedulerApp app_3 = getMockApplication(APPLICATION_ID++, user_0);
queue.submitApplication(app_3, user_0, A); queue.submitApplicationAttempt(app_3, user_0);
assertEquals(2, queue.getNumActiveApplications()); assertEquals(2, queue.getNumActiveApplications());
assertEquals(2, queue.getNumPendingApplications()); assertEquals(2, queue.getNumPendingApplications());
assertEquals(2, queue.getNumActiveApplications(user_0)); assertEquals(2, queue.getNumActiveApplications(user_0));
@ -420,7 +420,7 @@ public class TestApplicationLimits {
assertTrue(queue.pendingApplications.contains(app_3)); assertTrue(queue.pendingApplications.contains(app_3));
// Kill 3rd pending application // Kill 3rd pending application
queue.finishApplication(app_2, A); queue.finishApplicationAttempt(app_2, A);
assertEquals(2, queue.getNumActiveApplications()); assertEquals(2, queue.getNumActiveApplications());
assertEquals(1, queue.getNumPendingApplications()); assertEquals(1, queue.getNumPendingApplications());
assertEquals(2, queue.getNumActiveApplications(user_0)); assertEquals(2, queue.getNumActiveApplications(user_0));
@ -429,7 +429,7 @@ public class TestApplicationLimits {
assertFalse(queue.activeApplications.contains(app_2)); assertFalse(queue.activeApplications.contains(app_2));
// Finish 1st application, app_3 should become active // Finish 1st application, app_3 should become active
queue.finishApplication(app_0, A); queue.finishApplicationAttempt(app_0, A);
assertEquals(2, queue.getNumActiveApplications()); assertEquals(2, queue.getNumActiveApplications());
assertEquals(0, queue.getNumPendingApplications()); assertEquals(0, queue.getNumPendingApplications());
assertEquals(2, queue.getNumActiveApplications(user_0)); assertEquals(2, queue.getNumActiveApplications(user_0));
@ -439,7 +439,7 @@ public class TestApplicationLimits {
assertFalse(queue.activeApplications.contains(app_0)); assertFalse(queue.activeApplications.contains(app_0));
// Finish 2nd application // Finish 2nd application
queue.finishApplication(app_1, A); queue.finishApplicationAttempt(app_1, A);
assertEquals(1, queue.getNumActiveApplications()); assertEquals(1, queue.getNumActiveApplications());
assertEquals(0, queue.getNumPendingApplications()); assertEquals(0, queue.getNumPendingApplications());
assertEquals(1, queue.getNumActiveApplications(user_0)); assertEquals(1, queue.getNumActiveApplications(user_0));
@ -447,7 +447,7 @@ public class TestApplicationLimits {
assertFalse(queue.activeApplications.contains(app_1)); assertFalse(queue.activeApplications.contains(app_1));
// Finish 4th application // Finish 4th application
queue.finishApplication(app_3, A); queue.finishApplicationAttempt(app_3, A);
assertEquals(0, queue.getNumActiveApplications()); assertEquals(0, queue.getNumActiveApplications());
assertEquals(0, queue.getNumPendingApplications()); assertEquals(0, queue.getNumPendingApplications());
assertEquals(0, queue.getNumActiveApplications(user_0)); assertEquals(0, queue.getNumActiveApplications(user_0));
@ -507,7 +507,7 @@ public class TestApplicationLimits {
FiCaSchedulerApp app_0_0 = FiCaSchedulerApp app_0_0 =
spy(new FiCaSchedulerApp(appAttemptId_0_0, user_0, queue, spy(new FiCaSchedulerApp(appAttemptId_0_0, user_0, queue,
queue.getActiveUsersManager(), rmContext)); queue.getActiveUsersManager(), rmContext));
queue.submitApplication(app_0_0, user_0, A); queue.submitApplicationAttempt(app_0_0, user_0);
List<ResourceRequest> app_0_0_requests = new ArrayList<ResourceRequest>(); List<ResourceRequest> app_0_0_requests = new ArrayList<ResourceRequest>();
app_0_0_requests.add( app_0_0_requests.add(
@ -526,7 +526,7 @@ public class TestApplicationLimits {
FiCaSchedulerApp app_0_1 = FiCaSchedulerApp app_0_1 =
spy(new FiCaSchedulerApp(appAttemptId_0_1, user_0, queue, spy(new FiCaSchedulerApp(appAttemptId_0_1, user_0, queue,
queue.getActiveUsersManager(), rmContext)); queue.getActiveUsersManager(), rmContext));
queue.submitApplication(app_0_1, user_0, A); queue.submitApplicationAttempt(app_0_1, user_0);
List<ResourceRequest> app_0_1_requests = new ArrayList<ResourceRequest>(); List<ResourceRequest> app_0_1_requests = new ArrayList<ResourceRequest>();
app_0_1_requests.add( app_0_1_requests.add(
@ -545,7 +545,7 @@ public class TestApplicationLimits {
FiCaSchedulerApp app_1_0 = FiCaSchedulerApp app_1_0 =
spy(new FiCaSchedulerApp(appAttemptId_1_0, user_1, queue, spy(new FiCaSchedulerApp(appAttemptId_1_0, user_1, queue,
queue.getActiveUsersManager(), rmContext)); queue.getActiveUsersManager(), rmContext));
queue.submitApplication(app_1_0, user_1, A); queue.submitApplicationAttempt(app_1_0, user_1);
List<ResourceRequest> app_1_0_requests = new ArrayList<ResourceRequest>(); List<ResourceRequest> app_1_0_requests = new ArrayList<ResourceRequest>();
app_1_0_requests.add( app_1_0_requests.add(

View File

@ -64,7 +64,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManage
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
@ -555,9 +558,12 @@ public class TestCapacityScheduler {
ApplicationId appId = BuilderUtils.newApplicationId(100, 1); ApplicationId appId = BuilderUtils.newApplicationId(100, 1);
ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId( ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
appId, 1); appId, 1);
SchedulerEvent event = SchedulerEvent addAppEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId, "default", "user"); new AppAddedSchedulerEvent(appId, "default", "user");
cs.handle(event); cs.handle(addAppEvent);
SchedulerEvent addAttemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId);
cs.handle(addAttemptEvent);
// Verify the blacklist can be updated independent of requesting containers // Verify the blacklist can be updated independent of requesting containers
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
@ -596,10 +602,10 @@ public class TestCapacityScheduler {
public void testConcurrentAccessOnApplications() throws Exception { public void testConcurrentAccessOnApplications() throws Exception {
CapacityScheduler cs = new CapacityScheduler(); CapacityScheduler cs = new CapacityScheduler();
verifyConcurrentAccessOnApplications( verifyConcurrentAccessOnApplications(
cs.applications, FiCaSchedulerApp.class, Queue.class); cs.appAttempts, FiCaSchedulerApp.class, Queue.class);
} }
public static <T extends SchedulerApplication, Q extends Queue> public static <T extends SchedulerApplicationAttempt, Q extends Queue>
void verifyConcurrentAccessOnApplications( void verifyConcurrentAccessOnApplications(
final Map<ApplicationAttemptId, T> applications, Class<T> appClazz, final Map<ApplicationAttemptId, T> applications, Class<T> appClazz,
final Class<Q> queueClazz) final Class<Q> queueClazz)
@ -682,4 +688,21 @@ public class TestCapacityScheduler {
Assert.assertNull(scheduler.getAppsInQueue("nonexistentqueue")); Assert.assertNull(scheduler.getAppsInQueue("nonexistentqueue"));
} }
} @Test
public void testAddAndRemoveAppFromCapacityScheduler() throws Exception {
AsyncDispatcher rmDispatcher = new AsyncDispatcher();
CapacityScheduler cs = new CapacityScheduler();
CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
setupQueueConfiguration(conf);
cs.reinitialize(conf, new RMContextImpl(rmDispatcher, null, null, null,
null, null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM()));
SchedulerApplication app =
TestSchedulerUtils.verifyAppAddedAndRemovedFromScheduler(
cs.applications, cs, "a1");
Assert.assertEquals("a1", app.getQueue().getQueueName());
}
}

View File

@ -271,14 +271,14 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
new FiCaSchedulerApp(appAttemptId_0, user_0, a, new FiCaSchedulerApp(appAttemptId_0, user_0, a,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
a.submitApplication(app_0, user_0, B); a.submitApplicationAttempt(app_0, user_0);
final ApplicationAttemptId appAttemptId_1 = final ApplicationAttemptId appAttemptId_1 =
TestUtils.getMockApplicationAttemptId(1, 0); TestUtils.getMockApplicationAttemptId(1, 0);
FiCaSchedulerApp app_1 = FiCaSchedulerApp app_1 =
new FiCaSchedulerApp(appAttemptId_1, user_0, a, new FiCaSchedulerApp(appAttemptId_1, user_0, a,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
a.submitApplication(app_1, user_0, B); // same user a.submitApplicationAttempt(app_1, user_0); // same user
// Setup some nodes // Setup some nodes
@ -320,14 +320,14 @@ public class TestLeafQueue {
.getMockApplicationAttemptId(0, 1); .getMockApplicationAttemptId(0, 1);
FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_d, d, null, FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_d, d, null,
rmContext); rmContext);
d.submitApplication(app_0, user_d, D); d.submitApplicationAttempt(app_0, user_d);
// Attempt the same application again // Attempt the same application again
final ApplicationAttemptId appAttemptId_1 = TestUtils final ApplicationAttemptId appAttemptId_1 = TestUtils
.getMockApplicationAttemptId(0, 2); .getMockApplicationAttemptId(0, 2);
FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_d, d, null, FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_d, d, null,
rmContext); rmContext);
d.submitApplication(app_1, user_d, D); // same user d.submitApplicationAttempt(app_1, user_d); // same user
} }
@ -345,7 +345,7 @@ public class TestLeafQueue {
.getMockApplicationAttemptId(0, 1); .getMockApplicationAttemptId(0, 1);
FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a, null, FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a, null,
rmContext); rmContext);
a.submitApplication(app_0, user_0, B); a.submitApplicationAttempt(app_0, user_0);
when(cs.getApplication(appAttemptId_0)).thenReturn(app_0); when(cs.getApplication(appAttemptId_0)).thenReturn(app_0);
AppAttemptRemovedSchedulerEvent event = new AppAttemptRemovedSchedulerEvent( AppAttemptRemovedSchedulerEvent event = new AppAttemptRemovedSchedulerEvent(
@ -360,7 +360,7 @@ public class TestLeafQueue {
.getMockApplicationAttemptId(0, 2); .getMockApplicationAttemptId(0, 2);
FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_0, a, null, FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_0, a, null,
rmContext); rmContext);
a.submitApplication(app_1, user_0, B); // same user a.submitApplicationAttempt(app_1, user_0); // same user
assertEquals(1, a.getMetrics().getAppsSubmitted()); assertEquals(1, a.getMetrics().getAppsSubmitted());
assertEquals(1, a.getMetrics().getAppsPending()); assertEquals(1, a.getMetrics().getAppsPending());
@ -396,14 +396,14 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
new FiCaSchedulerApp(appAttemptId_0, user_0, a, new FiCaSchedulerApp(appAttemptId_0, user_0, a,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
a.submitApplication(app_0, user_0, A); a.submitApplicationAttempt(app_0, user_0);
final ApplicationAttemptId appAttemptId_1 = final ApplicationAttemptId appAttemptId_1 =
TestUtils.getMockApplicationAttemptId(1, 0); TestUtils.getMockApplicationAttemptId(1, 0);
FiCaSchedulerApp app_1 = FiCaSchedulerApp app_1 =
new FiCaSchedulerApp(appAttemptId_1, user_0, a, new FiCaSchedulerApp(appAttemptId_1, user_0, a,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
a.submitApplication(app_1, user_0, A); // same user a.submitApplicationAttempt(app_1, user_0); // same user
// Setup some nodes // Setup some nodes
@ -524,21 +524,21 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
new FiCaSchedulerApp(appAttemptId_0, user_0, a, new FiCaSchedulerApp(appAttemptId_0, user_0, a,
a.getActiveUsersManager(), rmContext); a.getActiveUsersManager(), rmContext);
a.submitApplication(app_0, user_0, A); a.submitApplicationAttempt(app_0, user_0);
final ApplicationAttemptId appAttemptId_1 = final ApplicationAttemptId appAttemptId_1 =
TestUtils.getMockApplicationAttemptId(1, 0); TestUtils.getMockApplicationAttemptId(1, 0);
FiCaSchedulerApp app_1 = FiCaSchedulerApp app_1 =
new FiCaSchedulerApp(appAttemptId_1, user_0, a, new FiCaSchedulerApp(appAttemptId_1, user_0, a,
a.getActiveUsersManager(), rmContext); a.getActiveUsersManager(), rmContext);
a.submitApplication(app_1, user_0, A); // same user a.submitApplicationAttempt(app_1, user_0); // same user
final ApplicationAttemptId appAttemptId_2 = final ApplicationAttemptId appAttemptId_2 =
TestUtils.getMockApplicationAttemptId(2, 0); TestUtils.getMockApplicationAttemptId(2, 0);
FiCaSchedulerApp app_2 = FiCaSchedulerApp app_2 =
new FiCaSchedulerApp(appAttemptId_2, user_1, a, new FiCaSchedulerApp(appAttemptId_2, user_1, a,
a.getActiveUsersManager(), rmContext); a.getActiveUsersManager(), rmContext);
a.submitApplication(app_2, user_1, A); a.submitApplicationAttempt(app_2, user_1);
// Setup some nodes // Setup some nodes
String host_0 = "127.0.0.1"; String host_0 = "127.0.0.1";
@ -618,21 +618,21 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
new FiCaSchedulerApp(appAttemptId_0, user_0, a, new FiCaSchedulerApp(appAttemptId_0, user_0, a,
a.getActiveUsersManager(), rmContext); a.getActiveUsersManager(), rmContext);
a.submitApplication(app_0, user_0, A); a.submitApplicationAttempt(app_0, user_0);
final ApplicationAttemptId appAttemptId_1 = final ApplicationAttemptId appAttemptId_1 =
TestUtils.getMockApplicationAttemptId(1, 0); TestUtils.getMockApplicationAttemptId(1, 0);
FiCaSchedulerApp app_1 = FiCaSchedulerApp app_1 =
new FiCaSchedulerApp(appAttemptId_1, user_0, a, new FiCaSchedulerApp(appAttemptId_1, user_0, a,
a.getActiveUsersManager(), rmContext); a.getActiveUsersManager(), rmContext);
a.submitApplication(app_1, user_0, A); // same user a.submitApplicationAttempt(app_1, user_0); // same user
final ApplicationAttemptId appAttemptId_2 = final ApplicationAttemptId appAttemptId_2 =
TestUtils.getMockApplicationAttemptId(2, 0); TestUtils.getMockApplicationAttemptId(2, 0);
FiCaSchedulerApp app_2 = FiCaSchedulerApp app_2 =
new FiCaSchedulerApp(appAttemptId_2, user_1, a, new FiCaSchedulerApp(appAttemptId_2, user_1, a,
a.getActiveUsersManager(), rmContext); a.getActiveUsersManager(), rmContext);
a.submitApplication(app_2, user_1, A); a.submitApplicationAttempt(app_2, user_1);
// Setup some nodes // Setup some nodes
String host_0 = "127.0.0.1"; String host_0 = "127.0.0.1";
@ -729,28 +729,28 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
new FiCaSchedulerApp(appAttemptId_0, user_0, a, new FiCaSchedulerApp(appAttemptId_0, user_0, a,
a.getActiveUsersManager(), rmContext); a.getActiveUsersManager(), rmContext);
a.submitApplication(app_0, user_0, A); a.submitApplicationAttempt(app_0, user_0);
final ApplicationAttemptId appAttemptId_1 = final ApplicationAttemptId appAttemptId_1 =
TestUtils.getMockApplicationAttemptId(1, 0); TestUtils.getMockApplicationAttemptId(1, 0);
FiCaSchedulerApp app_1 = FiCaSchedulerApp app_1 =
new FiCaSchedulerApp(appAttemptId_1, user_0, a, new FiCaSchedulerApp(appAttemptId_1, user_0, a,
a.getActiveUsersManager(), rmContext); a.getActiveUsersManager(), rmContext);
a.submitApplication(app_1, user_0, A); // same user a.submitApplicationAttempt(app_1, user_0); // same user
final ApplicationAttemptId appAttemptId_2 = final ApplicationAttemptId appAttemptId_2 =
TestUtils.getMockApplicationAttemptId(2, 0); TestUtils.getMockApplicationAttemptId(2, 0);
FiCaSchedulerApp app_2 = FiCaSchedulerApp app_2 =
new FiCaSchedulerApp(appAttemptId_2, user_1, a, new FiCaSchedulerApp(appAttemptId_2, user_1, a,
a.getActiveUsersManager(), rmContext); a.getActiveUsersManager(), rmContext);
a.submitApplication(app_2, user_1, A); a.submitApplicationAttempt(app_2, user_1);
final ApplicationAttemptId appAttemptId_3 = final ApplicationAttemptId appAttemptId_3 =
TestUtils.getMockApplicationAttemptId(3, 0); TestUtils.getMockApplicationAttemptId(3, 0);
FiCaSchedulerApp app_3 = FiCaSchedulerApp app_3 =
new FiCaSchedulerApp(appAttemptId_3, user_2, a, new FiCaSchedulerApp(appAttemptId_3, user_2, a,
a.getActiveUsersManager(), rmContext); a.getActiveUsersManager(), rmContext);
a.submitApplication(app_3, user_2, A); a.submitApplicationAttempt(app_3, user_2);
// Setup some nodes // Setup some nodes
String host_0 = "127.0.0.1"; String host_0 = "127.0.0.1";
@ -905,14 +905,14 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
new FiCaSchedulerApp(appAttemptId_0, user_0, a, new FiCaSchedulerApp(appAttemptId_0, user_0, a,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
a.submitApplication(app_0, user_0, A); a.submitApplicationAttempt(app_0, user_0);
final ApplicationAttemptId appAttemptId_1 = final ApplicationAttemptId appAttemptId_1 =
TestUtils.getMockApplicationAttemptId(1, 0); TestUtils.getMockApplicationAttemptId(1, 0);
FiCaSchedulerApp app_1 = FiCaSchedulerApp app_1 =
new FiCaSchedulerApp(appAttemptId_1, user_1, a, new FiCaSchedulerApp(appAttemptId_1, user_1, a,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
a.submitApplication(app_1, user_1, A); a.submitApplicationAttempt(app_1, user_1);
// Setup some nodes // Setup some nodes
String host_0 = "127.0.0.1"; String host_0 = "127.0.0.1";
@ -1007,14 +1007,14 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
new FiCaSchedulerApp(appAttemptId_0, user_0, a, new FiCaSchedulerApp(appAttemptId_0, user_0, a,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
a.submitApplication(app_0, user_0, A); a.submitApplicationAttempt(app_0, user_0);
final ApplicationAttemptId appAttemptId_1 = final ApplicationAttemptId appAttemptId_1 =
TestUtils.getMockApplicationAttemptId(1, 0); TestUtils.getMockApplicationAttemptId(1, 0);
FiCaSchedulerApp app_1 = FiCaSchedulerApp app_1 =
new FiCaSchedulerApp(appAttemptId_1, user_1, a, new FiCaSchedulerApp(appAttemptId_1, user_1, a,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
a.submitApplication(app_1, user_1, A); a.submitApplicationAttempt(app_1, user_1);
// Setup some nodes // Setup some nodes
String host_0 = "127.0.0.1"; String host_0 = "127.0.0.1";
@ -1111,14 +1111,14 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
new FiCaSchedulerApp(appAttemptId_0, user_0, a, new FiCaSchedulerApp(appAttemptId_0, user_0, a,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
a.submitApplication(app_0, user_0, A); a.submitApplicationAttempt(app_0, user_0);
final ApplicationAttemptId appAttemptId_1 = final ApplicationAttemptId appAttemptId_1 =
TestUtils.getMockApplicationAttemptId(1, 0); TestUtils.getMockApplicationAttemptId(1, 0);
FiCaSchedulerApp app_1 = FiCaSchedulerApp app_1 =
new FiCaSchedulerApp(appAttemptId_1, user_1, a, new FiCaSchedulerApp(appAttemptId_1, user_1, a,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
a.submitApplication(app_1, user_1, A); a.submitApplicationAttempt(app_1, user_1);
// Setup some nodes // Setup some nodes
String host_0 = "127.0.0.1"; String host_0 = "127.0.0.1";
@ -1232,7 +1232,7 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a,
mock(ActiveUsersManager.class), rmContext)); mock(ActiveUsersManager.class), rmContext));
a.submitApplication(app_0, user_0, A); a.submitApplicationAttempt(app_0, user_0);
// Setup some nodes and racks // Setup some nodes and racks
String host_0 = "127.0.0.1"; String host_0 = "127.0.0.1";
@ -1373,7 +1373,7 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a,
mock(ActiveUsersManager.class), rmContext)); mock(ActiveUsersManager.class), rmContext));
a.submitApplication(app_0, user_0, A); a.submitApplicationAttempt(app_0, user_0);
// Setup some nodes and racks // Setup some nodes and racks
String host_0 = "127.0.0.1"; String host_0 = "127.0.0.1";
@ -1504,7 +1504,7 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a,
mock(ActiveUsersManager.class), rmContext)); mock(ActiveUsersManager.class), rmContext));
a.submitApplication(app_0, user_0, A); a.submitApplicationAttempt(app_0, user_0);
// Setup some nodes and racks // Setup some nodes and racks
String host_0_0 = "127.0.0.1"; String host_0_0 = "127.0.0.1";
@ -1607,21 +1607,21 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
new FiCaSchedulerApp(appAttemptId_0, user_e, e, new FiCaSchedulerApp(appAttemptId_0, user_e, e,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
e.submitApplication(app_0, user_e, E); e.submitApplicationAttempt(app_0, user_e);
final ApplicationAttemptId appAttemptId_1 = final ApplicationAttemptId appAttemptId_1 =
TestUtils.getMockApplicationAttemptId(1, 0); TestUtils.getMockApplicationAttemptId(1, 0);
FiCaSchedulerApp app_1 = FiCaSchedulerApp app_1 =
new FiCaSchedulerApp(appAttemptId_1, user_e, e, new FiCaSchedulerApp(appAttemptId_1, user_e, e,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
e.submitApplication(app_1, user_e, E); // same user e.submitApplicationAttempt(app_1, user_e); // same user
final ApplicationAttemptId appAttemptId_2 = final ApplicationAttemptId appAttemptId_2 =
TestUtils.getMockApplicationAttemptId(2, 0); TestUtils.getMockApplicationAttemptId(2, 0);
FiCaSchedulerApp app_2 = FiCaSchedulerApp app_2 =
new FiCaSchedulerApp(appAttemptId_2, user_e, e, new FiCaSchedulerApp(appAttemptId_2, user_e, e,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
e.submitApplication(app_2, user_e, E); // same user e.submitApplicationAttempt(app_2, user_e); // same user
// before reinitialization // before reinitialization
assertEquals(2, e.activeApplications.size()); assertEquals(2, e.activeApplications.size());
@ -1685,21 +1685,21 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
new FiCaSchedulerApp(appAttemptId_0, user_e, e, new FiCaSchedulerApp(appAttemptId_0, user_e, e,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
e.submitApplication(app_0, user_e, E); e.submitApplicationAttempt(app_0, user_e);
final ApplicationAttemptId appAttemptId_1 = final ApplicationAttemptId appAttemptId_1 =
TestUtils.getMockApplicationAttemptId(1, 0); TestUtils.getMockApplicationAttemptId(1, 0);
FiCaSchedulerApp app_1 = FiCaSchedulerApp app_1 =
new FiCaSchedulerApp(appAttemptId_1, user_e, e, new FiCaSchedulerApp(appAttemptId_1, user_e, e,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
e.submitApplication(app_1, user_e, E); // same user e.submitApplicationAttempt(app_1, user_e); // same user
final ApplicationAttemptId appAttemptId_2 = final ApplicationAttemptId appAttemptId_2 =
TestUtils.getMockApplicationAttemptId(2, 0); TestUtils.getMockApplicationAttemptId(2, 0);
FiCaSchedulerApp app_2 = FiCaSchedulerApp app_2 =
new FiCaSchedulerApp(appAttemptId_2, user_e, e, new FiCaSchedulerApp(appAttemptId_2, user_e, e,
mock(ActiveUsersManager.class), rmContext); mock(ActiveUsersManager.class), rmContext);
e.submitApplication(app_2, user_e, E); // same user e.submitApplicationAttempt(app_2, user_e); // same user
// before updating cluster resource // before updating cluster resource
assertEquals(2, e.activeApplications.size()); assertEquals(2, e.activeApplications.size());
@ -1762,14 +1762,14 @@ public class TestLeafQueue {
FiCaSchedulerApp app_0 = FiCaSchedulerApp app_0 =
spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a, spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a,
mock(ActiveUsersManager.class), rmContext)); mock(ActiveUsersManager.class), rmContext));
a.submitApplication(app_0, user_0, A); a.submitApplicationAttempt(app_0, user_0);
final ApplicationAttemptId appAttemptId_1 = final ApplicationAttemptId appAttemptId_1 =
TestUtils.getMockApplicationAttemptId(1, 0); TestUtils.getMockApplicationAttemptId(1, 0);
FiCaSchedulerApp app_1 = FiCaSchedulerApp app_1 =
spy(new FiCaSchedulerApp(appAttemptId_1, user_0, a, spy(new FiCaSchedulerApp(appAttemptId_1, user_0, a,
mock(ActiveUsersManager.class), rmContext)); mock(ActiveUsersManager.class), rmContext));
a.submitApplication(app_1, user_0, A); a.submitApplicationAttempt(app_1, user_0);
// Setup some nodes and racks // Setup some nodes and racks
String host_0_0 = "127.0.0.1"; String host_0_0 = "127.0.0.1";

View File

@ -78,7 +78,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
@ -255,7 +257,12 @@ public class TestFairScheduler {
private ApplicationAttemptId createSchedulingRequest(int memory, int vcores, private ApplicationAttemptId createSchedulingRequest(int memory, int vcores,
String queueId, String userId, int numContainers, int priority) { String queueId, String userId, int numContainers, int priority) {
ApplicationAttemptId id = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); ApplicationAttemptId id = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
scheduler.addApplicationAttempt(id, queueId, userId); scheduler.addApplication(id.getApplicationId(), queueId, userId);
// This conditional is for testAclSubmitApplication where app is rejected
// and no app is added.
if (scheduler.applications.containsKey(id.getApplicationId())) {
scheduler.addApplicationAttempt(id);
}
List<ResourceRequest> ask = new ArrayList<ResourceRequest>(); List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
ResourceRequest request = createResourceRequest(memory, vcores, ResourceRequest.ANY, ResourceRequest request = createResourceRequest(memory, vcores, ResourceRequest.ANY,
priority, numContainers, true); priority, numContainers, true);
@ -583,7 +590,7 @@ public class TestFairScheduler {
// Make sure queue 2 is waiting with a reservation // Make sure queue 2 is waiting with a reservation
assertEquals(0, scheduler.getQueueManager().getQueue("queue2"). assertEquals(0, scheduler.getQueueManager().getQueue("queue2").
getResourceUsage().getMemory()); getResourceUsage().getMemory());
assertEquals(1024, scheduler.applications.get(attId).getCurrentReservation().getMemory()); assertEquals(1024, scheduler.appAttempts.get(attId).getCurrentReservation().getMemory());
// Now another node checks in with capacity // Now another node checks in with capacity
RMNode node2 = RMNode node2 =
@ -599,10 +606,10 @@ public class TestFairScheduler {
getResourceUsage().getMemory()); getResourceUsage().getMemory());
// The old reservation should still be there... // The old reservation should still be there...
assertEquals(1024, scheduler.applications.get(attId).getCurrentReservation().getMemory()); assertEquals(1024, scheduler.appAttempts.get(attId).getCurrentReservation().getMemory());
// ... but it should disappear when we update the first node. // ... but it should disappear when we update the first node.
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
assertEquals(0, scheduler.applications.get(attId).getCurrentReservation().getMemory()); assertEquals(0, scheduler.appAttempts.get(attId).getCurrentReservation().getMemory());
} }
@ -618,9 +625,13 @@ public class TestFairScheduler {
null, null, null, false, false, 0, null, null), null, null, 0, null); null, null, null, false, false, 0, null, null), null, null, 0, null);
appsMap.put(appAttemptId.getApplicationId(), rmApp); appsMap.put(appAttemptId.getApplicationId(), rmApp);
AppAttemptAddedSchedulerEvent appAddedEvent = AppAddedSchedulerEvent appAddedEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId, "default", "user1"); new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "default",
"user1");
scheduler.handle(appAddedEvent); scheduler.handle(appAddedEvent);
AppAttemptAddedSchedulerEvent attempAddedEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId);
scheduler.handle(attempAddedEvent);
assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true) assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true)
.getRunnableAppSchedulables().size()); .getRunnableAppSchedulables().size());
assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true) assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true)
@ -640,9 +651,13 @@ public class TestFairScheduler {
null, null, null, false, false, 0, null, null), null, null, 0, null); null, null, null, false, false, 0, null, null), null, null, 0, null);
appsMap.put(appAttemptId.getApplicationId(), rmApp); appsMap.put(appAttemptId.getApplicationId(), rmApp);
AppAttemptAddedSchedulerEvent appAddedEvent2 = AppAddedSchedulerEvent appAddedEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId, "default", "user2"); new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "default",
scheduler.handle(appAddedEvent2); "user2");
scheduler.handle(appAddedEvent);
AppAttemptAddedSchedulerEvent attempAddedEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId);
scheduler.handle(attempAddedEvent);
assertEquals(0, scheduler.getQueueManager().getLeafQueue("user1", true) assertEquals(0, scheduler.getQueueManager().getLeafQueue("user1", true)
.getRunnableAppSchedulables().size()); .getRunnableAppSchedulables().size());
assertEquals(1, scheduler.getQueueManager().getLeafQueue("default", true) assertEquals(1, scheduler.getQueueManager().getLeafQueue("default", true)
@ -660,8 +675,8 @@ public class TestFairScheduler {
// submit app with empty queue // submit app with empty queue
ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1); ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1);
AppAttemptAddedSchedulerEvent appAddedEvent = AppAddedSchedulerEvent appAddedEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId, "", "user1"); new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "", "user1");
scheduler.handle(appAddedEvent); scheduler.handle(appAddedEvent);
// submission rejected // submission rejected
@ -695,7 +710,7 @@ public class TestFairScheduler {
scheduler.reinitialize(conf, resourceManager.getRMContext()); scheduler.reinitialize(conf, resourceManager.getRMContext());
ApplicationAttemptId appId; ApplicationAttemptId appId;
Map<ApplicationAttemptId, FSSchedulerApp> apps = scheduler.applications; Map<ApplicationAttemptId, FSSchedulerApp> apps = scheduler.appAttempts;
List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>(); List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
rules.add(new QueuePlacementRule.Specified().initialize(true, null)); rules.add(new QueuePlacementRule.Specified().initialize(true, null));
@ -786,11 +801,14 @@ public class TestFairScheduler {
scheduler.reinitialize(conf, resourceManager.getRMContext()); scheduler.reinitialize(conf, resourceManager.getRMContext());
ApplicationAttemptId id11 = createAppAttemptId(1, 1); ApplicationAttemptId id11 = createAppAttemptId(1, 1);
scheduler.addApplicationAttempt(id11, "root.queue1", "user1"); scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1");
scheduler.addApplicationAttempt(id11);
ApplicationAttemptId id21 = createAppAttemptId(2, 1); ApplicationAttemptId id21 = createAppAttemptId(2, 1);
scheduler.addApplicationAttempt(id21, "root.queue2", "user1"); scheduler.addApplication(id21.getApplicationId(), "root.queue2", "user1");
scheduler.addApplicationAttempt(id21);
ApplicationAttemptId id22 = createAppAttemptId(2, 2); ApplicationAttemptId id22 = createAppAttemptId(2, 2);
scheduler.addApplicationAttempt(id22, "root.queue2", "user1"); scheduler.addApplication(id22.getApplicationId(), "root.queue2", "user1");
scheduler.addApplicationAttempt(id22);
int minReqSize = int minReqSize =
FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB; FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB;
@ -831,11 +849,13 @@ public class TestFairScheduler {
@Test @Test
public void testAppAdditionAndRemoval() throws Exception { public void testAppAdditionAndRemoval() throws Exception {
scheduler.reinitialize(conf, resourceManager.getRMContext()); scheduler.reinitialize(conf, resourceManager.getRMContext());
ApplicationAttemptId attemptId =createAppAttemptId(1, 1);
AppAttemptAddedSchedulerEvent appAddedEvent1 = AppAddedSchedulerEvent appAddedEvent = new AppAddedSchedulerEvent(attemptId.getApplicationId(), "default",
new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), "default",
"user1"); "user1");
scheduler.handle(appAddedEvent1); scheduler.handle(appAddedEvent);
AppAttemptAddedSchedulerEvent attemptAddedEvent =
new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1));
scheduler.handle(attemptAddedEvent);
// Scheduler should have two queues (the default and the one created for user1) // Scheduler should have two queues (the default and the one created for user1)
assertEquals(2, scheduler.getQueueManager().getLeafQueues().size()); assertEquals(2, scheduler.getQueueManager().getLeafQueues().size());
@ -1118,12 +1138,12 @@ public class TestFairScheduler {
scheduler.handle(nodeUpdate3); scheduler.handle(nodeUpdate3);
} }
assertEquals(1, scheduler.applications.get(app1).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size());
assertEquals(1, scheduler.applications.get(app2).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app2).getLiveContainers().size());
assertEquals(1, scheduler.applications.get(app3).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app3).getLiveContainers().size());
assertEquals(1, scheduler.applications.get(app4).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size());
assertEquals(1, scheduler.applications.get(app5).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app5).getLiveContainers().size());
assertEquals(1, scheduler.applications.get(app6).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app6).getLiveContainers().size());
// Now new requests arrive from queues C and D // Now new requests arrive from queues C and D
ApplicationAttemptId app7 = ApplicationAttemptId app7 =
@ -1146,16 +1166,16 @@ public class TestFairScheduler {
// Make sure it is lowest priority container. // Make sure it is lowest priority container.
scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(), scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
Resources.createResource(2 * 1024)); Resources.createResource(2 * 1024));
assertEquals(1, scheduler.applications.get(app1).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size());
assertEquals(1, scheduler.applications.get(app2).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app2).getLiveContainers().size());
assertEquals(1, scheduler.applications.get(app4).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size());
assertEquals(1, scheduler.applications.get(app5).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app5).getLiveContainers().size());
// First verify we are adding containers to preemption list for the application // First verify we are adding containers to preemption list for the application
assertTrue(!Collections.disjoint(scheduler.applications.get(app3).getLiveContainers(), assertTrue(!Collections.disjoint(scheduler.appAttempts.get(app3).getLiveContainers(),
scheduler.applications.get(app3).getPreemptionContainers())); scheduler.appAttempts.get(app3).getPreemptionContainers()));
assertTrue(!Collections.disjoint(scheduler.applications.get(app6).getLiveContainers(), assertTrue(!Collections.disjoint(scheduler.appAttempts.get(app6).getLiveContainers(),
scheduler.applications.get(app6).getPreemptionContainers())); scheduler.appAttempts.get(app6).getPreemptionContainers()));
// Pretend 15 seconds have passed // Pretend 15 seconds have passed
clock.tick(15); clock.tick(15);
@ -1165,8 +1185,8 @@ public class TestFairScheduler {
Resources.createResource(2 * 1024)); Resources.createResource(2 * 1024));
// At this point the containers should have been killed (since we are not simulating AM) // At this point the containers should have been killed (since we are not simulating AM)
assertEquals(0, scheduler.applications.get(app6).getLiveContainers().size()); assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size());
assertEquals(0, scheduler.applications.get(app3).getLiveContainers().size()); assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size());
// Trigger a kill by insisting we want containers back // Trigger a kill by insisting we want containers back
scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(), scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
@ -1180,22 +1200,22 @@ public class TestFairScheduler {
scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(), scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
Resources.createResource(2 * 1024)); Resources.createResource(2 * 1024));
assertEquals(1, scheduler.applications.get(app1).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size());
assertEquals(0, scheduler.applications.get(app2).getLiveContainers().size()); assertEquals(0, scheduler.appAttempts.get(app2).getLiveContainers().size());
assertEquals(0, scheduler.applications.get(app3).getLiveContainers().size()); assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size());
assertEquals(1, scheduler.applications.get(app4).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size());
assertEquals(0, scheduler.applications.get(app5).getLiveContainers().size()); assertEquals(0, scheduler.appAttempts.get(app5).getLiveContainers().size());
assertEquals(0, scheduler.applications.get(app6).getLiveContainers().size()); assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size());
// Now A and B are below fair share, so preemption shouldn't do anything // Now A and B are below fair share, so preemption shouldn't do anything
scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(), scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(),
Resources.createResource(2 * 1024)); Resources.createResource(2 * 1024));
assertEquals(1, scheduler.applications.get(app1).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size());
assertEquals(0, scheduler.applications.get(app2).getLiveContainers().size()); assertEquals(0, scheduler.appAttempts.get(app2).getLiveContainers().size());
assertEquals(0, scheduler.applications.get(app3).getLiveContainers().size()); assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size());
assertEquals(1, scheduler.applications.get(app4).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size());
assertEquals(0, scheduler.applications.get(app5).getLiveContainers().size()); assertEquals(0, scheduler.appAttempts.get(app5).getLiveContainers().size());
assertEquals(0, scheduler.applications.get(app6).getLiveContainers().size()); assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size());
} }
@Test (timeout = 5000) @Test (timeout = 5000)
@ -1354,9 +1374,9 @@ public class TestFairScheduler {
// One container should get reservation and the other should get nothing // One container should get reservation and the other should get nothing
assertEquals(1024, assertEquals(1024,
scheduler.applications.get(attId1).getCurrentReservation().getMemory()); scheduler.appAttempts.get(attId1).getCurrentReservation().getMemory());
assertEquals(0, assertEquals(0,
scheduler.applications.get(attId2).getCurrentReservation().getMemory()); scheduler.appAttempts.get(attId2).getCurrentReservation().getMemory());
} }
@Test (timeout = 5000) @Test (timeout = 5000)
@ -1391,7 +1411,7 @@ public class TestFairScheduler {
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
// App 1 should be running // App 1 should be running
assertEquals(1, scheduler.applications.get(attId1).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(attId1).getLiveContainers().size());
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1", ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
"user1", 1); "user1", 1);
@ -1400,7 +1420,7 @@ public class TestFairScheduler {
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
// App 2 should not be running // App 2 should not be running
assertEquals(0, scheduler.applications.get(attId2).getLiveContainers().size()); assertEquals(0, scheduler.appAttempts.get(attId2).getLiveContainers().size());
// Request another container for app 1 // Request another container for app 1
createSchedulingRequestExistingApplication(1024, 1, attId1); createSchedulingRequestExistingApplication(1024, 1, attId1);
@ -1409,7 +1429,7 @@ public class TestFairScheduler {
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
// Request should be fulfilled // Request should be fulfilled
assertEquals(2, scheduler.applications.get(attId1).getLiveContainers().size()); assertEquals(2, scheduler.appAttempts.get(attId1).getLiveContainers().size());
} }
@Test (timeout = 5000) @Test (timeout = 5000)
@ -1429,10 +1449,10 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1); NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
FSSchedulerApp app = scheduler.applications.get(attId); FSSchedulerApp app = scheduler.appAttempts.get(attId);
assertEquals(1, app.getLiveContainers().size()); assertEquals(1, app.getLiveContainers().size());
ContainerId containerId = scheduler.applications.get(attId) ContainerId containerId = scheduler.appAttempts.get(attId)
.getLiveContainers().iterator().next().getContainerId(); .getLiveContainers().iterator().next().getContainerId();
// Cause reservation to be created // Cause reservation to be created
@ -1501,9 +1521,9 @@ public class TestFairScheduler {
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1", ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
"norealuserhasthisname2", 1); "norealuserhasthisname2", 1);
FSSchedulerApp app1 = scheduler.applications.get(attId1); FSSchedulerApp app1 = scheduler.appAttempts.get(attId1);
assertNotNull("The application was not allowed", app1); assertNotNull("The application was not allowed", app1);
FSSchedulerApp app2 = scheduler.applications.get(attId2); FSSchedulerApp app2 = scheduler.appAttempts.get(attId2);
assertNull("The application was allowed", app2); assertNull("The application was allowed", app2);
} }
@ -1526,7 +1546,8 @@ public class TestFairScheduler {
scheduler.handle(nodeEvent2); scheduler.handle(nodeEvent2);
ApplicationAttemptId appId = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); ApplicationAttemptId appId = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
scheduler.addApplicationAttempt(appId, "queue1", "user1"); scheduler.addApplication(appId.getApplicationId(), "queue1", "user1");
scheduler.addApplicationAttempt(appId);
// 1 request with 2 nodes on the same rack. another request with 1 node on // 1 request with 2 nodes on the same rack. another request with 1 node on
// a different rack // a different rack
@ -1545,14 +1566,14 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent updateEvent1 = new NodeUpdateSchedulerEvent(node1); NodeUpdateSchedulerEvent updateEvent1 = new NodeUpdateSchedulerEvent(node1);
scheduler.handle(updateEvent1); scheduler.handle(updateEvent1);
// should assign node local // should assign node local
assertEquals(1, scheduler.applications.get(appId).getLiveContainers().size()); assertEquals(1, scheduler.appAttempts.get(appId).getLiveContainers().size());
// node 2 checks in // node 2 checks in
scheduler.update(); scheduler.update();
NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2); NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
scheduler.handle(updateEvent2); scheduler.handle(updateEvent2);
// should assign rack local // should assign rack local
assertEquals(2, scheduler.applications.get(appId).getLiveContainers().size()); assertEquals(2, scheduler.appAttempts.get(appId).getLiveContainers().size());
} }
@Test (timeout = 5000) @Test (timeout = 5000)
@ -1571,8 +1592,8 @@ public class TestFairScheduler {
"user1", 2); "user1", 2);
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1", ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
"user1", 2); "user1", 2);
FSSchedulerApp app1 = scheduler.applications.get(attId1); FSSchedulerApp app1 = scheduler.appAttempts.get(attId1);
FSSchedulerApp app2 = scheduler.applications.get(attId2); FSSchedulerApp app2 = scheduler.appAttempts.get(attId2);
FSLeafQueue queue1 = scheduler.getQueueManager().getLeafQueue("queue1", true); FSLeafQueue queue1 = scheduler.getQueueManager().getLeafQueue("queue1", true);
queue1.setPolicy(new FifoPolicy()); queue1.setPolicy(new FifoPolicy());
@ -1612,7 +1633,7 @@ public class TestFairScheduler {
ApplicationAttemptId attId = ApplicationAttemptId attId =
createSchedulingRequest(1024, "root.default", "user", 8); createSchedulingRequest(1024, "root.default", "user", 8);
FSSchedulerApp app = scheduler.applications.get(attId); FSSchedulerApp app = scheduler.appAttempts.get(attId);
// set maxAssign to 2: only 2 containers should be allocated // set maxAssign to 2: only 2 containers should be allocated
scheduler.maxAssign = 2; scheduler.maxAssign = 2;
@ -1674,10 +1695,10 @@ public class TestFairScheduler {
ApplicationAttemptId attId4 = ApplicationAttemptId attId4 =
createSchedulingRequest(1024, fifoQueue, user, 4); createSchedulingRequest(1024, fifoQueue, user, 4);
FSSchedulerApp app1 = scheduler.applications.get(attId1); FSSchedulerApp app1 = scheduler.appAttempts.get(attId1);
FSSchedulerApp app2 = scheduler.applications.get(attId2); FSSchedulerApp app2 = scheduler.appAttempts.get(attId2);
FSSchedulerApp app3 = scheduler.applications.get(attId3); FSSchedulerApp app3 = scheduler.appAttempts.get(attId3);
FSSchedulerApp app4 = scheduler.applications.get(attId4); FSSchedulerApp app4 = scheduler.appAttempts.get(attId4);
scheduler.getQueueManager().getLeafQueue(fifoQueue, true) scheduler.getQueueManager().getLeafQueue(fifoQueue, true)
.setPolicy(SchedulingPolicy.parse("fifo")); .setPolicy(SchedulingPolicy.parse("fifo"));
@ -1764,7 +1785,7 @@ public class TestFairScheduler {
ApplicationAttemptId attId = ApplicationAttemptId attId =
ApplicationAttemptId.newInstance(applicationId, this.ATTEMPT_ID++); ApplicationAttemptId.newInstance(applicationId, this.ATTEMPT_ID++);
scheduler.addApplicationAttempt(attId, queue, user); scheduler.addApplication(attId.getApplicationId(), queue, user);
numTries = 0; numTries = 0;
while (application.getFinishTime() == 0 && numTries < MAX_TRIES) { while (application.getFinishTime() == 0 && numTries < MAX_TRIES) {
@ -1792,7 +1813,7 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1); NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
FSSchedulerApp app = scheduler.applications.get(attId); FSSchedulerApp app = scheduler.appAttempts.get(attId);
assertEquals(0, app.getLiveContainers().size()); assertEquals(0, app.getLiveContainers().size());
assertEquals(0, app.getReservedContainers().size()); assertEquals(0, app.getReservedContainers().size());
@ -1861,7 +1882,7 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2); NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2);
// no matter how many heartbeats, node2 should never get a container // no matter how many heartbeats, node2 should never get a container
FSSchedulerApp app = scheduler.applications.get(attId1); FSSchedulerApp app = scheduler.appAttempts.get(attId1);
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
scheduler.handle(node2UpdateEvent); scheduler.handle(node2UpdateEvent);
assertEquals(0, app.getLiveContainers().size()); assertEquals(0, app.getLiveContainers().size());
@ -1900,7 +1921,7 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2); NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2);
// no matter how many heartbeats, node2 should never get a container // no matter how many heartbeats, node2 should never get a container
FSSchedulerApp app = scheduler.applications.get(attId1); FSSchedulerApp app = scheduler.appAttempts.get(attId1);
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
scheduler.handle(node2UpdateEvent); scheduler.handle(node2UpdateEvent);
assertEquals(0, app.getLiveContainers().size()); assertEquals(0, app.getLiveContainers().size());
@ -1933,7 +1954,7 @@ public class TestFairScheduler {
ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1", ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1",
"user1", 0); "user1", 0);
FSSchedulerApp app = scheduler.applications.get(attId); FSSchedulerApp app = scheduler.appAttempts.get(attId);
ResourceRequest nodeRequest = createResourceRequest(1024, node2.getHostName(), 1, 2, true); ResourceRequest nodeRequest = createResourceRequest(1024, node2.getHostName(), 1, 2, true);
ResourceRequest rackRequest = createResourceRequest(1024, "rack1", 1, 2, true); ResourceRequest rackRequest = createResourceRequest(1024, "rack1", 1, 2, true);
@ -1973,7 +1994,7 @@ public class TestFairScheduler {
ApplicationAttemptId attId = createSchedulingRequest(1024, 1, "default", ApplicationAttemptId attId = createSchedulingRequest(1024, 1, "default",
"user1", 2); "user1", 2);
FSSchedulerApp app = scheduler.applications.get(attId); FSSchedulerApp app = scheduler.appAttempts.get(attId);
scheduler.update(); scheduler.update();
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1); NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
@ -1993,10 +2014,10 @@ public class TestFairScheduler {
ApplicationAttemptId appAttId1 = createSchedulingRequest(2048, 1, "queue1", ApplicationAttemptId appAttId1 = createSchedulingRequest(2048, 1, "queue1",
"user1", 2); "user1", 2);
FSSchedulerApp app1 = scheduler.applications.get(appAttId1); FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1);
ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 2, "queue1", ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 2, "queue1",
"user1", 2); "user1", 2);
FSSchedulerApp app2 = scheduler.applications.get(appAttId2); FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2);
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy(); DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
drfPolicy.initialize(scheduler.getClusterCapacity()); drfPolicy.initialize(scheduler.getClusterCapacity());
@ -2034,13 +2055,13 @@ public class TestFairScheduler {
ApplicationAttemptId appAttId1 = createSchedulingRequest(3072, 1, "queue1", ApplicationAttemptId appAttId1 = createSchedulingRequest(3072, 1, "queue1",
"user1", 2); "user1", 2);
FSSchedulerApp app1 = scheduler.applications.get(appAttId1); FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1);
ApplicationAttemptId appAttId2 = createSchedulingRequest(2048, 2, "queue1", ApplicationAttemptId appAttId2 = createSchedulingRequest(2048, 2, "queue1",
"user1", 2); "user1", 2);
FSSchedulerApp app2 = scheduler.applications.get(appAttId2); FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2);
ApplicationAttemptId appAttId3 = createSchedulingRequest(1024, 2, "queue2", ApplicationAttemptId appAttId3 = createSchedulingRequest(1024, 2, "queue2",
"user1", 2); "user1", 2);
FSSchedulerApp app3 = scheduler.applications.get(appAttId3); FSSchedulerApp app3 = scheduler.appAttempts.get(appAttId3);
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy(); DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
drfPolicy.initialize(scheduler.getClusterCapacity()); drfPolicy.initialize(scheduler.getClusterCapacity());
@ -2071,19 +2092,19 @@ public class TestFairScheduler {
ApplicationAttemptId appAttId1 = createSchedulingRequest(3074, 1, "queue1.subqueue1", ApplicationAttemptId appAttId1 = createSchedulingRequest(3074, 1, "queue1.subqueue1",
"user1", 2); "user1", 2);
Thread.sleep(3); // so that start times will be different Thread.sleep(3); // so that start times will be different
FSSchedulerApp app1 = scheduler.applications.get(appAttId1); FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1);
ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 3, "queue1.subqueue1", ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 3, "queue1.subqueue1",
"user1", 2); "user1", 2);
Thread.sleep(3); // so that start times will be different Thread.sleep(3); // so that start times will be different
FSSchedulerApp app2 = scheduler.applications.get(appAttId2); FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2);
ApplicationAttemptId appAttId3 = createSchedulingRequest(2048, 2, "queue1.subqueue2", ApplicationAttemptId appAttId3 = createSchedulingRequest(2048, 2, "queue1.subqueue2",
"user1", 2); "user1", 2);
Thread.sleep(3); // so that start times will be different Thread.sleep(3); // so that start times will be different
FSSchedulerApp app3 = scheduler.applications.get(appAttId3); FSSchedulerApp app3 = scheduler.appAttempts.get(appAttId3);
ApplicationAttemptId appAttId4 = createSchedulingRequest(1024, 2, "queue2", ApplicationAttemptId appAttId4 = createSchedulingRequest(1024, 2, "queue2",
"user1", 2); "user1", 2);
Thread.sleep(3); // so that start times will be different Thread.sleep(3); // so that start times will be different
FSSchedulerApp app4 = scheduler.applications.get(appAttId4); FSSchedulerApp app4 = scheduler.appAttempts.get(appAttId4);
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy(); DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
drfPolicy.initialize(scheduler.getClusterCapacity()); drfPolicy.initialize(scheduler.getClusterCapacity());
@ -2163,7 +2184,7 @@ public class TestFairScheduler {
NodeUpdateSchedulerEvent(node2); NodeUpdateSchedulerEvent(node2);
// no matter how many heartbeats, node2 should never get a container // no matter how many heartbeats, node2 should never get a container
FSSchedulerApp app = scheduler.applications.get(attId1); FSSchedulerApp app = scheduler.appAttempts.get(attId1);
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
scheduler.handle(node2UpdateEvent); scheduler.handle(node2UpdateEvent);
assertEquals(0, app.getLiveContainers().size()); assertEquals(0, app.getLiveContainers().size());
@ -2178,12 +2199,12 @@ public class TestFairScheduler {
public void testConcurrentAccessOnApplications() throws Exception { public void testConcurrentAccessOnApplications() throws Exception {
FairScheduler fs = new FairScheduler(); FairScheduler fs = new FairScheduler();
TestCapacityScheduler.verifyConcurrentAccessOnApplications( TestCapacityScheduler.verifyConcurrentAccessOnApplications(
fs.applications, FSSchedulerApp.class, FSLeafQueue.class); fs.appAttempts, FSSchedulerApp.class, FSLeafQueue.class);
} }
private void verifyAppRunnable(ApplicationAttemptId attId, boolean runnable) { private void verifyAppRunnable(ApplicationAttemptId attId, boolean runnable) {
FSSchedulerApp app = scheduler.applications.get(attId); FSSchedulerApp app = scheduler.appAttempts.get(attId);
FSLeafQueue queue = app.getQueue(); FSLeafQueue queue = app.getQueue();
Collection<AppSchedulable> runnableApps = Collection<AppSchedulable> runnableApps =
queue.getRunnableAppSchedulables(); queue.getRunnableAppSchedulables();
@ -2356,7 +2377,8 @@ public class TestFairScheduler {
// send application request // send application request
ApplicationAttemptId appAttemptId = ApplicationAttemptId appAttemptId =
createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
fs.addApplicationAttempt(appAttemptId, "queue11", "user11"); fs.addApplication(appAttemptId.getApplicationId(), "queue11", "user11");
fs.addApplicationAttempt(appAttemptId);
List<ResourceRequest> ask = new ArrayList<ResourceRequest>(); List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
ResourceRequest request = ResourceRequest request =
createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true); createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
@ -2367,7 +2389,7 @@ public class TestFairScheduler {
// at least one pass // at least one pass
Thread.sleep(fs.getConf().getContinuousSchedulingSleepMs() + 500); Thread.sleep(fs.getConf().getContinuousSchedulingSleepMs() + 500);
FSSchedulerApp app = fs.applications.get(appAttemptId); FSSchedulerApp app = fs.appAttempts.get(appAttemptId);
// Wait until app gets resources. // Wait until app gets resources.
while (app.getCurrentConsumption().equals(Resources.none())) { } while (app.getCurrentConsumption().equals(Resources.none())) { }
@ -2455,7 +2477,7 @@ public class TestFairScheduler {
ApplicationAttemptId appAttemptId = ApplicationAttemptId appAttemptId =
createSchedulingRequest(GB, "root.default", "user", 1); createSchedulingRequest(GB, "root.default", "user", 1);
FSSchedulerApp app = scheduler.applications.get(appAttemptId); FSSchedulerApp app = scheduler.appAttempts.get(appAttemptId);
// Verify the blacklist can be updated independent of requesting containers // Verify the blacklist can be updated independent of requesting containers
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
@ -2465,7 +2487,7 @@ public class TestFairScheduler {
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
Collections.<ContainerId>emptyList(), null, Collections.<ContainerId>emptyList(), null,
Collections.singletonList(host)); Collections.singletonList(host));
assertFalse(scheduler.applications.get(appAttemptId).isBlacklisted(host)); assertFalse(scheduler.appAttempts.get(appAttemptId).isBlacklisted(host));
List<ResourceRequest> update = Arrays.asList( List<ResourceRequest> update = Arrays.asList(
createResourceRequest(GB, node.getHostName(), 1, 0, true)); createResourceRequest(GB, node.getHostName(), 1, 0, true));
@ -2527,4 +2549,12 @@ public class TestFairScheduler {
assertTrue(appAttIds.contains(appAttId1)); assertTrue(appAttIds.contains(appAttId1));
assertTrue(appAttIds.contains(appAttId2)); assertTrue(appAttIds.contains(appAttId2));
} }
@Test
public void testAddAndRemoveAppFromFairScheduler() throws Exception {
FairScheduler scheduler =
(FairScheduler) resourceManager.getResourceScheduler();
TestSchedulerUtils.verifyAppAddedAndRemovedFromScheduler(
scheduler.applications, scheduler, "default");
}
} }

View File

@ -61,13 +61,16 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
@ -150,14 +153,21 @@ public class TestFifoScheduler {
ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId( ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
appId, 1); appId, 1);
SchedulerEvent event = SchedulerEvent appEvent = new AppAddedSchedulerEvent(appId, "queue", "user");
new AppAttemptAddedSchedulerEvent(appAttemptId, "queue", "user"); schedular.handle(appEvent);
schedular.handle(event); SchedulerEvent attemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId);
schedular.handle(attemptEvent);
appAttemptId = BuilderUtils.newApplicationAttemptId(appId, 2); appAttemptId = BuilderUtils.newApplicationAttemptId(appId, 2);
event = new AppAttemptAddedSchedulerEvent(appAttemptId, "queue", "user"); SchedulerEvent appEvent2 =
schedular.handle(event); new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "queue",
"user");
schedular.handle(appEvent2);
SchedulerEvent attemptEvent2 =
new AppAttemptAddedSchedulerEvent(appAttemptId);
schedular.handle(attemptEvent2);
int afterAppsSubmitted = metrics.getAppsSubmitted(); int afterAppsSubmitted = metrics.getAppsSubmitted();
Assert.assertEquals(1, afterAppsSubmitted - beforeAppsSubmitted); Assert.assertEquals(1, afterAppsSubmitted - beforeAppsSubmitted);
@ -188,9 +198,13 @@ public class TestFifoScheduler {
int _appAttemptId = 1; int _appAttemptId = 1;
ApplicationAttemptId appAttemptId = createAppAttemptId(_appId, ApplicationAttemptId appAttemptId = createAppAttemptId(_appId,
_appAttemptId); _appAttemptId);
AppAttemptAddedSchedulerEvent appEvent1 = AppAddedSchedulerEvent appEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId, "queue1", "user1"); new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "queue1",
scheduler.handle(appEvent1); "user1");
scheduler.handle(appEvent);
AppAttemptAddedSchedulerEvent attemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId);
scheduler.handle(attemptEvent);
int memory = 64; int memory = 64;
int nConts = 3; int nConts = 3;
@ -274,9 +288,13 @@ public class TestFifoScheduler {
int _appAttemptId = 1; int _appAttemptId = 1;
ApplicationAttemptId appAttemptId = createAppAttemptId(_appId, ApplicationAttemptId appAttemptId = createAppAttemptId(_appId,
_appAttemptId); _appAttemptId);
AppAttemptAddedSchedulerEvent appEvent1 = AppAddedSchedulerEvent appEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId, "queue1", "user1"); new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "queue1",
scheduler.handle(appEvent1); "user1");
scheduler.handle(appEvent);
AppAttemptAddedSchedulerEvent attemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId);
scheduler.handle(attemptEvent);
int memory = 1024; int memory = 1024;
int priority = 1; int priority = 1;
@ -520,7 +538,7 @@ public class TestFifoScheduler {
public void testConcurrentAccessOnApplications() throws Exception { public void testConcurrentAccessOnApplications() throws Exception {
FifoScheduler fs = new FifoScheduler(); FifoScheduler fs = new FifoScheduler();
TestCapacityScheduler.verifyConcurrentAccessOnApplications( TestCapacityScheduler.verifyConcurrentAccessOnApplications(
fs.applications, FiCaSchedulerApp.class, Queue.class); fs.appAttempts, FiCaSchedulerApp.class, Queue.class);
} }
@SuppressWarnings("resource") @SuppressWarnings("resource")
@ -541,9 +559,13 @@ public class TestFifoScheduler {
ApplicationId appId = BuilderUtils.newApplicationId(100, 1); ApplicationId appId = BuilderUtils.newApplicationId(100, 1);
ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId( ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
appId, 1); appId, 1);
SchedulerEvent event = SchedulerEvent appEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId, "default", "user"); new AppAddedSchedulerEvent(appId, "default",
fs.handle(event); "user");
fs.handle(appEvent);
SchedulerEvent attemptEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId);
fs.handle(attemptEvent);
// Verify the blacklist can be updated independent of requesting containers // Verify the blacklist can be updated independent of requesting containers
fs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), fs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
@ -575,6 +597,17 @@ public class TestFifoScheduler {
Assert.assertNull(scheduler.getAppsInQueue("someotherqueue")); Assert.assertNull(scheduler.getAppsInQueue("someotherqueue"));
} }
@Test
public void testAddAndRemoveAppFromFiFoScheduler() throws Exception {
Configuration conf = new Configuration();
conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class,
ResourceScheduler.class);
MockRM rm = new MockRM(conf);
FifoScheduler fs = (FifoScheduler)rm.getResourceScheduler();
TestSchedulerUtils.verifyAppAddedAndRemovedFromScheduler(fs.applications,
fs, "queue");
}
private void checkApplicationResourceUsage(int expected, private void checkApplicationResourceUsage(int expected,
Application application) { Application application) {
Assert.assertEquals(expected, application.getUsedResources().getMemory()); Assert.assertEquals(expected, application.getUsedResources().getMemory());

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager; import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
@ -1392,6 +1393,8 @@ public class TestRMWebServicesApps extends JerseyTest {
MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(CONTAINER_MB, "testwordcount", "user1"); RMApp app1 = rm.submitApp(CONTAINER_MB, "testwordcount", "user1");
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
rm.waitForState(app1.getCurrentAppAttempt().getAppAttemptId(),
RMAppAttemptState.ALLOCATED);
int maxAppAttempts = rm.getConfig().getInt( int maxAppAttempts = rm.getConfig().getInt(
YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
@ -1405,6 +1408,8 @@ public class TestRMWebServicesApps extends JerseyTest {
rm.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); rm.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
} }
rm.waitForState(app1.getCurrentAppAttempt().getAppAttemptId(),
RMAppAttemptState.ALLOCATED);
assertEquals("incorrect number of attempts", maxAppAttempts, assertEquals("incorrect number of attempts", maxAppAttempts,
app1.getAppAttempts().values().size()); app1.getAppAttempts().values().size());
testAppAttemptsHelper(app1.getApplicationId().toString(), app1, testAppAttemptsHelper(app1.getApplicationId().toString(), app1,