YARN-1490. Introduced the ability to make ResourceManager optionally not kill all containers when an ApplicationMaster exits. Contributed by Jian He.
svn merge --ignore-ancestry -c 1557143 ../../trunk/ git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1557144 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
9619fc7740
commit
76fc12eadf
|
@ -861,5 +861,10 @@ public class ResourceSchedulerWrapper implements ResourceScheduler,
|
||||||
public List<ApplicationAttemptId> getAppsInQueue(String queue) {
|
public List<ApplicationAttemptId> getAppsInQueue(String queue) {
|
||||||
return scheduler.getAppsInQueue(queue);
|
return scheduler.getAppsInQueue(queue);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RMContainer getRMContainer(ContainerId containerId) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -40,6 +40,9 @@ Release 2.4.0 - UNRELEASED
|
||||||
YARN-1029. Added embedded leader election in the ResourceManager. (Karthik
|
YARN-1029. Added embedded leader election in the ResourceManager. (Karthik
|
||||||
Kambatla via vinodkv)
|
Kambatla via vinodkv)
|
||||||
|
|
||||||
|
YARN-1490. Introduced the ability to make ResourceManager optionally not kill
|
||||||
|
all containers when an ApplicationMaster exits. (Jian He via vinodkv)
|
||||||
|
|
||||||
IMPROVEMENTS
|
IMPROVEMENTS
|
||||||
|
|
||||||
YARN-7. Support CPU resource for DistributedShell. (Junping Du via llu)
|
YARN-7. Support CPU resource for DistributedShell. (Junping Du via llu)
|
||||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
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.ApplicationClientProtocol;
|
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
|
||||||
|
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
|
||||||
import org.apache.hadoop.yarn.util.Records;
|
import org.apache.hadoop.yarn.util.Records;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -57,7 +58,8 @@ public abstract class ApplicationSubmissionContext {
|
||||||
ApplicationId applicationId, String applicationName, String queue,
|
ApplicationId applicationId, String applicationName, String queue,
|
||||||
Priority priority, ContainerLaunchContext amContainer,
|
Priority priority, ContainerLaunchContext amContainer,
|
||||||
boolean isUnmanagedAM, boolean cancelTokensWhenComplete,
|
boolean isUnmanagedAM, boolean cancelTokensWhenComplete,
|
||||||
int maxAppAttempts, Resource resource, String applicationType) {
|
int maxAppAttempts, Resource resource, String applicationType,
|
||||||
|
boolean keepContainers) {
|
||||||
ApplicationSubmissionContext context =
|
ApplicationSubmissionContext context =
|
||||||
Records.newRecord(ApplicationSubmissionContext.class);
|
Records.newRecord(ApplicationSubmissionContext.class);
|
||||||
context.setApplicationId(applicationId);
|
context.setApplicationId(applicationId);
|
||||||
|
@ -70,9 +72,22 @@ public abstract class ApplicationSubmissionContext {
|
||||||
context.setMaxAppAttempts(maxAppAttempts);
|
context.setMaxAppAttempts(maxAppAttempts);
|
||||||
context.setResource(resource);
|
context.setResource(resource);
|
||||||
context.setApplicationType(applicationType);
|
context.setApplicationType(applicationType);
|
||||||
|
context.setKeepContainersAcrossApplicationAttempts(keepContainers);
|
||||||
return context;
|
return context;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Public
|
||||||
|
@Stable
|
||||||
|
public static ApplicationSubmissionContext newInstance(
|
||||||
|
ApplicationId applicationId, String applicationName, String queue,
|
||||||
|
Priority priority, ContainerLaunchContext amContainer,
|
||||||
|
boolean isUnmanagedAM, boolean cancelTokensWhenComplete,
|
||||||
|
int maxAppAttempts, Resource resource, String applicationType) {
|
||||||
|
return newInstance(applicationId, applicationName, queue, priority,
|
||||||
|
amContainer, isUnmanagedAM, cancelTokensWhenComplete, maxAppAttempts,
|
||||||
|
resource, null, false);
|
||||||
|
}
|
||||||
|
|
||||||
@Public
|
@Public
|
||||||
@Stable
|
@Stable
|
||||||
public static ApplicationSubmissionContext newInstance(
|
public static ApplicationSubmissionContext newInstance(
|
||||||
|
@ -268,4 +283,35 @@ public abstract class ApplicationSubmissionContext {
|
||||||
@Public
|
@Public
|
||||||
@Stable
|
@Stable
|
||||||
public abstract void setApplicationType(String applicationType);
|
public abstract void setApplicationType(String applicationType);
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the flag which indicates whether to keep containers across application
|
||||||
|
* attempts or not.
|
||||||
|
*
|
||||||
|
* @return the flag which indicates whether to keep containers across
|
||||||
|
* application attempts or not.
|
||||||
|
*/
|
||||||
|
@Public
|
||||||
|
@Stable
|
||||||
|
public abstract boolean getKeepContainersAcrossApplicationAttempts();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the flag which indicates whether to keep containers across application
|
||||||
|
* attempts.
|
||||||
|
* <p>
|
||||||
|
* If the flag is true, running containers will not be killed when application
|
||||||
|
* attempt fails and these containers will be retrieved by the new application
|
||||||
|
* attempt on registration via
|
||||||
|
* {@link ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)}.
|
||||||
|
* </p>
|
||||||
|
*
|
||||||
|
* @param keepContainers
|
||||||
|
* the flag which indicates whether to keep containers across
|
||||||
|
* application attempts.
|
||||||
|
*/
|
||||||
|
@Public
|
||||||
|
@Stable
|
||||||
|
public abstract void setKeepContainersAcrossApplicationAttempts(
|
||||||
|
boolean keepContainers);
|
||||||
}
|
}
|
|
@ -46,10 +46,20 @@ public abstract class ContainerId implements Comparable<ContainerId>{
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the <code>ApplicationAttemptId</code> of the application to which
|
* Get the <code>ApplicationAttemptId</code> of the application to which the
|
||||||
* the <code>Container</code> was assigned.
|
* <code>Container</code> was assigned.
|
||||||
* @return <code>ApplicationAttemptId</code> of the application to which
|
* <p>
|
||||||
* the <code>Container</code> was assigned
|
* Note: If containers are kept alive across application attempts via
|
||||||
|
* {@link ApplicationSubmissionContext#setKeepContainersAcrossApplicationAttempts(boolean)}
|
||||||
|
* the <code>ContainerId</code> does not necessarily contain the current
|
||||||
|
* running application attempt's <code>ApplicationAttemptId</code> This
|
||||||
|
* container can be allocated by previously exited application attempt and
|
||||||
|
* managed by the current running attempt thus have the previous application
|
||||||
|
* attempt's <code>ApplicationAttemptId</code>.
|
||||||
|
* </p>
|
||||||
|
*
|
||||||
|
* @return <code>ApplicationAttemptId</code> of the application to which the
|
||||||
|
* <code>Container</code> was assigned
|
||||||
*/
|
*/
|
||||||
@Public
|
@Public
|
||||||
@Stable
|
@Stable
|
||||||
|
|
|
@ -248,6 +248,7 @@ message ApplicationSubmissionContextProto {
|
||||||
optional int32 maxAppAttempts = 8 [default = 0];
|
optional int32 maxAppAttempts = 8 [default = 0];
|
||||||
optional ResourceProto resource = 9;
|
optional ResourceProto resource = 9;
|
||||||
optional string applicationType = 10 [default = "YARN"];
|
optional string applicationType = 10 [default = "YARN"];
|
||||||
|
optional bool keep_containers_across_application_attempts = 11 [default = false];
|
||||||
}
|
}
|
||||||
|
|
||||||
enum ApplicationAccessTypeProto {
|
enum ApplicationAccessTypeProto {
|
||||||
|
|
|
@ -298,6 +298,19 @@ extends ApplicationSubmissionContext {
|
||||||
this.resource = resource;
|
this.resource = resource;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void
|
||||||
|
setKeepContainersAcrossApplicationAttempts(boolean keepContainers) {
|
||||||
|
maybeInitBuilder();
|
||||||
|
builder.setKeepContainersAcrossApplicationAttempts(keepContainers);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean getKeepContainersAcrossApplicationAttempts() {
|
||||||
|
ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
|
||||||
|
return p.getKeepContainersAcrossApplicationAttempts();
|
||||||
|
}
|
||||||
|
|
||||||
private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
|
private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
|
||||||
return new PriorityPBImpl(p);
|
return new PriorityPBImpl(p);
|
||||||
}
|
}
|
||||||
|
|
|
@ -422,11 +422,18 @@ public class ApplicationMasterService extends AbstractService implements
|
||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
RMApp app =
|
||||||
RMServerUtils.validateContainerReleaseRequest(release, appAttemptId);
|
this.rmContext.getRMApps().get(appAttemptId.getApplicationId());
|
||||||
} catch (InvalidContainerReleaseException e) {
|
// In the case of work-preserving AM restart, it's possible for the
|
||||||
LOG.warn("Invalid container release by application " + appAttemptId, e);
|
// AM to release containers from the earlier attempt.
|
||||||
throw e;
|
if (!app.getApplicationSubmissionContext()
|
||||||
|
.getKeepContainersAcrossApplicationAttempts()) {
|
||||||
|
try {
|
||||||
|
RMServerUtils.validateContainerReleaseRequest(release, appAttemptId);
|
||||||
|
} catch (InvalidContainerReleaseException e) {
|
||||||
|
LOG.warn("Invalid container release by application " + appAttemptId, e);
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Send new requests to appAttempt.
|
// Send new requests to appAttempt.
|
||||||
|
@ -434,8 +441,6 @@ public class ApplicationMasterService extends AbstractService implements
|
||||||
this.rScheduler.allocate(appAttemptId, ask, release,
|
this.rScheduler.allocate(appAttemptId, ask, release,
|
||||||
blacklistAdditions, blacklistRemovals);
|
blacklistAdditions, blacklistRemovals);
|
||||||
|
|
||||||
RMApp app = this.rmContext.getRMApps().get(
|
|
||||||
appAttemptId.getApplicationId());
|
|
||||||
RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId);
|
RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId);
|
||||||
|
|
||||||
AllocateResponse allocateResponse =
|
AllocateResponse allocateResponse =
|
||||||
|
|
|
@ -23,14 +23,20 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
public class RMAppFailedAttemptEvent extends RMAppEvent {
|
public class RMAppFailedAttemptEvent extends RMAppEvent {
|
||||||
|
|
||||||
private final String diagnostics;
|
private final String diagnostics;
|
||||||
|
private final boolean transferStateFromPreviousAttempt;
|
||||||
|
|
||||||
public RMAppFailedAttemptEvent(ApplicationId appId, RMAppEventType event,
|
public RMAppFailedAttemptEvent(ApplicationId appId, RMAppEventType event,
|
||||||
String diagnostics) {
|
String diagnostics, boolean transferStateFromPreviousAttempt) {
|
||||||
super(appId, event);
|
super(appId, event);
|
||||||
this.diagnostics = diagnostics;
|
this.diagnostics = diagnostics;
|
||||||
|
this.transferStateFromPreviousAttempt = transferStateFromPreviousAttempt;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getDiagnostics() {
|
public String getDiagnostics() {
|
||||||
return this.diagnostics;
|
return this.diagnostics;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean getTransferStateFromPreviousAttempt() {
|
||||||
|
return transferStateFromPreviousAttempt;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -63,6 +63,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
|
||||||
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.RMAppAttemptImpl;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
|
||||||
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.RMAppStartAttemptEvent;
|
||||||
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;
|
||||||
|
@ -76,6 +77,7 @@ import org.apache.hadoop.yarn.state.StateMachine;
|
||||||
import org.apache.hadoop.yarn.state.StateMachineFactory;
|
import org.apache.hadoop.yarn.state.StateMachineFactory;
|
||||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||||
|
|
||||||
|
@SuppressWarnings({ "rawtypes", "unchecked" })
|
||||||
public class RMAppImpl implements RMApp, Recoverable {
|
public class RMAppImpl implements RMApp, Recoverable {
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(RMAppImpl.class);
|
private static final Log LOG = LogFactory.getLog(RMAppImpl.class);
|
||||||
|
@ -646,24 +648,26 @@ public class RMAppImpl implements RMApp, Recoverable {
|
||||||
|
|
||||||
for(int i=0; i<appState.getAttemptCount(); ++i) {
|
for(int i=0; i<appState.getAttemptCount(); ++i) {
|
||||||
// create attempt
|
// create attempt
|
||||||
createNewAttempt(false);
|
createNewAttempt();
|
||||||
((RMAppAttemptImpl)this.currentAttempt).recover(state);
|
((RMAppAttemptImpl)this.currentAttempt).recover(state);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
private void createNewAttempt() {
|
||||||
private void createNewAttempt(boolean startAttempt) {
|
|
||||||
ApplicationAttemptId appAttemptId =
|
ApplicationAttemptId appAttemptId =
|
||||||
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);
|
submissionContext, conf, maxAppAttempts == attempts.size());
|
||||||
attempts.put(appAttemptId, attempt);
|
attempts.put(appAttemptId, attempt);
|
||||||
currentAttempt = attempt;
|
currentAttempt = attempt;
|
||||||
if(startAttempt) {
|
}
|
||||||
handler.handle(
|
|
||||||
new RMAppAttemptEvent(appAttemptId, RMAppAttemptEventType.START));
|
private void
|
||||||
}
|
createAndStartNewAttempt(boolean transferStateFromPreviousAttempt) {
|
||||||
|
createNewAttempt();
|
||||||
|
handler.handle(new RMAppStartAttemptEvent(currentAttempt.getAppAttemptId(),
|
||||||
|
transferStateFromPreviousAttempt));
|
||||||
}
|
}
|
||||||
|
|
||||||
private void processNodeUpdate(RMAppNodeUpdateType type, RMNode node) {
|
private void processNodeUpdate(RMAppNodeUpdateType type, RMNode node) {
|
||||||
|
@ -688,7 +692,6 @@ public class RMAppImpl implements RMApp, Recoverable {
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
private static final class RMAppRecoveredTransition implements
|
private static final class RMAppRecoveredTransition implements
|
||||||
MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
|
MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
|
||||||
|
|
||||||
|
@ -729,7 +732,6 @@ public class RMAppImpl implements RMApp, Recoverable {
|
||||||
|
|
||||||
private static final class AddApplicationToSchedulerTransition extends
|
private static final class AddApplicationToSchedulerTransition extends
|
||||||
RMAppTransition {
|
RMAppTransition {
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
@Override
|
@Override
|
||||||
public void transition(RMAppImpl app, RMAppEvent event) {
|
public void transition(RMAppImpl app, RMAppEvent event) {
|
||||||
if (event instanceof RMAppNewSavedEvent) {
|
if (event instanceof RMAppNewSavedEvent) {
|
||||||
|
@ -751,14 +753,13 @@ public class RMAppImpl implements RMApp, Recoverable {
|
||||||
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) {
|
||||||
app.createNewAttempt(true);
|
app.createAndStartNewAttempt(false);
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final class FinalStateSavedTransition implements
|
private static final class FinalStateSavedTransition implements
|
||||||
MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
|
MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
|
||||||
|
|
||||||
@SuppressWarnings({ "rawtypes", "unchecked" })
|
|
||||||
@Override
|
@Override
|
||||||
public RMAppState transition(RMAppImpl app, RMAppEvent event) {
|
public RMAppState transition(RMAppImpl app, RMAppEvent event) {
|
||||||
RMAppUpdateSavedEvent storeEvent = (RMAppUpdateSavedEvent) event;
|
RMAppUpdateSavedEvent storeEvent = (RMAppUpdateSavedEvent) event;
|
||||||
|
@ -959,7 +960,6 @@ public class RMAppImpl implements RMApp, Recoverable {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class KillAttemptTransition extends RMAppTransition {
|
private static class KillAttemptTransition extends RMAppTransition {
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
@Override
|
@Override
|
||||||
public void transition(RMAppImpl app, RMAppEvent event) {
|
public void transition(RMAppImpl app, RMAppEvent event) {
|
||||||
app.stateBeforeKilling = app.getState();
|
app.stateBeforeKilling = app.getState();
|
||||||
|
@ -987,7 +987,6 @@ public class RMAppImpl implements RMApp, Recoverable {
|
||||||
return nodes;
|
return nodes;
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
public void transition(RMAppImpl app, RMAppEvent event) {
|
public void transition(RMAppImpl app, RMAppEvent event) {
|
||||||
Set<NodeId> nodes = getNodesOnWhichAttemptRan(app);
|
Set<NodeId> nodes = getNodesOnWhichAttemptRan(app);
|
||||||
for (NodeId nodeId : nodes) {
|
for (NodeId nodeId : nodes) {
|
||||||
|
@ -1019,7 +1018,21 @@ public class RMAppImpl implements RMApp, Recoverable {
|
||||||
public RMAppState transition(RMAppImpl app, RMAppEvent event) {
|
public RMAppState transition(RMAppImpl app, RMAppEvent event) {
|
||||||
if (!app.submissionContext.getUnmanagedAM()
|
if (!app.submissionContext.getUnmanagedAM()
|
||||||
&& app.attempts.size() < app.maxAppAttempts) {
|
&& app.attempts.size() < app.maxAppAttempts) {
|
||||||
app.createNewAttempt(true);
|
boolean transferStateFromPreviousAttempt = false;
|
||||||
|
RMAppFailedAttemptEvent failedEvent = (RMAppFailedAttemptEvent) event;
|
||||||
|
transferStateFromPreviousAttempt =
|
||||||
|
failedEvent.getTransferStateFromPreviousAttempt();
|
||||||
|
|
||||||
|
RMAppAttempt oldAttempt = app.currentAttempt;
|
||||||
|
app.createAndStartNewAttempt(transferStateFromPreviousAttempt);
|
||||||
|
// Transfer the state from the previous attempt to the current attempt.
|
||||||
|
// Note that the previous failed attempt may still be collecting the
|
||||||
|
// container events from the scheduler and update its data structures
|
||||||
|
// before the new attempt is created.
|
||||||
|
if (transferStateFromPreviousAttempt) {
|
||||||
|
((RMAppAttemptImpl) app.currentAttempt)
|
||||||
|
.transferStateFromPreviousAttempt(oldAttempt);
|
||||||
|
}
|
||||||
return initialState;
|
return initialState;
|
||||||
} else {
|
} else {
|
||||||
app.rememberTargetTransitionsAndStoreState(event,
|
app.rememberTargetTransitionsAndStoreState(event,
|
||||||
|
|
|
@ -129,9 +129,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
||||||
private SecretKey clientTokenMasterKey = null;
|
private SecretKey clientTokenMasterKey = null;
|
||||||
|
|
||||||
//nodes on while this attempt's containers ran
|
//nodes on while this attempt's containers ran
|
||||||
private final Set<NodeId> ranNodes =
|
private Set<NodeId> ranNodes =
|
||||||
new HashSet<NodeId>();
|
new HashSet<NodeId>();
|
||||||
private final List<ContainerStatus> justFinishedContainers =
|
private List<ContainerStatus> justFinishedContainers =
|
||||||
new ArrayList<ContainerStatus>();
|
new ArrayList<ContainerStatus>();
|
||||||
private Container masterContainer;
|
private Container masterContainer;
|
||||||
|
|
||||||
|
@ -148,7 +148,7 @@ 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 final boolean isLastAttempt;
|
||||||
private static final ExpiredTransition EXPIRED_TRANSITION =
|
private static final ExpiredTransition EXPIRED_TRANSITION =
|
||||||
new ExpiredTransition();
|
new ExpiredTransition();
|
||||||
|
|
||||||
|
@ -330,6 +330,12 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
||||||
RMAppAttemptEventType.KILL))
|
RMAppAttemptEventType.KILL))
|
||||||
|
|
||||||
// Transitions from FAILED State
|
// Transitions from FAILED State
|
||||||
|
// For work-preserving AM restart, failed attempt are still capturing
|
||||||
|
// CONTAINER_FINISHED event and record the finished containers for the
|
||||||
|
// use by the next new attempt.
|
||||||
|
.addTransition(RMAppAttemptState.FAILED, RMAppAttemptState.FAILED,
|
||||||
|
RMAppAttemptEventType.CONTAINER_FINISHED,
|
||||||
|
new ContainerFinishedAtFailedTransition())
|
||||||
.addTransition(
|
.addTransition(
|
||||||
RMAppAttemptState.FAILED,
|
RMAppAttemptState.FAILED,
|
||||||
RMAppAttemptState.FAILED,
|
RMAppAttemptState.FAILED,
|
||||||
|
@ -338,8 +344,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
||||||
RMAppAttemptEventType.KILL,
|
RMAppAttemptEventType.KILL,
|
||||||
RMAppAttemptEventType.UNREGISTERED,
|
RMAppAttemptEventType.UNREGISTERED,
|
||||||
RMAppAttemptEventType.STATUS_UPDATE,
|
RMAppAttemptEventType.STATUS_UPDATE,
|
||||||
RMAppAttemptEventType.CONTAINER_ALLOCATED,
|
RMAppAttemptEventType.CONTAINER_ALLOCATED))
|
||||||
RMAppAttemptEventType.CONTAINER_FINISHED))
|
|
||||||
|
|
||||||
// Transitions from FINISHING State
|
// Transitions from FINISHING State
|
||||||
.addTransition(RMAppAttemptState.FINISHING,
|
.addTransition(RMAppAttemptState.FINISHING,
|
||||||
|
@ -390,7 +395,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) {
|
Configuration conf, boolean isLastAttempt) {
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
this.applicationAttemptId = appAttemptId;
|
this.applicationAttemptId = appAttemptId;
|
||||||
this.rmContext = rmContext;
|
this.rmContext = rmContext;
|
||||||
|
@ -404,7 +409,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
||||||
this.writeLock = lock.writeLock();
|
this.writeLock = lock.writeLock();
|
||||||
|
|
||||||
this.proxiedTrackingUrl = generateProxyUriWithScheme(null);
|
this.proxiedTrackingUrl = generateProxyUriWithScheme(null);
|
||||||
|
this.isLastAttempt = isLastAttempt;
|
||||||
this.stateMachine = stateMachineFactory.make(this);
|
this.stateMachine = stateMachineFactory.make(this);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -685,6 +690,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
||||||
this.startTime = attemptState.getStartTime();
|
this.startTime = attemptState.getStartTime();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void transferStateFromPreviousAttempt(RMAppAttempt attempt) {
|
||||||
|
this.justFinishedContainers = attempt.getJustFinishedContainers();
|
||||||
|
this.ranNodes = attempt.getRanNodes();
|
||||||
|
}
|
||||||
|
|
||||||
private void recoverAppAttemptCredentials(Credentials appAttemptTokens)
|
private void recoverAppAttemptCredentials(Credentials appAttemptTokens)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (appAttemptTokens == null) {
|
if (appAttemptTokens == null) {
|
||||||
|
@ -721,6 +731,12 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
||||||
public void transition(RMAppAttemptImpl appAttempt,
|
public void transition(RMAppAttemptImpl appAttempt,
|
||||||
RMAppAttemptEvent event) {
|
RMAppAttemptEvent event) {
|
||||||
|
|
||||||
|
boolean transferStateFromPreviousAttempt = false;
|
||||||
|
if (event instanceof RMAppStartAttemptEvent) {
|
||||||
|
transferStateFromPreviousAttempt =
|
||||||
|
((RMAppStartAttemptEvent) event)
|
||||||
|
.getTransferStateFromPreviousAttempt();
|
||||||
|
}
|
||||||
appAttempt.startTime = System.currentTimeMillis();
|
appAttempt.startTime = System.currentTimeMillis();
|
||||||
|
|
||||||
// Register with the ApplicationMasterService
|
// Register with the ApplicationMasterService
|
||||||
|
@ -740,9 +756,10 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
||||||
new Token<AMRMTokenIdentifier>(id,
|
new Token<AMRMTokenIdentifier>(id,
|
||||||
appAttempt.rmContext.getAMRMTokenSecretManager());
|
appAttempt.rmContext.getAMRMTokenSecretManager());
|
||||||
|
|
||||||
// Add the applicationAttempt to the scheduler
|
// Add the applicationAttempt to the scheduler and inform the scheduler
|
||||||
|
// whether to transfer the state from previous attempt.
|
||||||
appAttempt.eventHandler.handle(new AppAttemptAddedSchedulerEvent(
|
appAttempt.eventHandler.handle(new AppAttemptAddedSchedulerEvent(
|
||||||
appAttempt.applicationAttemptId));
|
appAttempt.applicationAttemptId, transferStateFromPreviousAttempt));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -981,6 +998,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
||||||
// Tell the application and the scheduler
|
// Tell the application and the scheduler
|
||||||
ApplicationId applicationId = appAttemptId.getApplicationId();
|
ApplicationId applicationId = appAttemptId.getApplicationId();
|
||||||
RMAppEvent appEvent = null;
|
RMAppEvent appEvent = null;
|
||||||
|
boolean keepContainersAcrossAppAttempts = false;
|
||||||
switch (finalAttemptState) {
|
switch (finalAttemptState) {
|
||||||
case FINISHED:
|
case FINISHED:
|
||||||
{
|
{
|
||||||
|
@ -996,7 +1014,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
||||||
appEvent =
|
appEvent =
|
||||||
new RMAppFailedAttemptEvent(applicationId,
|
new RMAppFailedAttemptEvent(applicationId,
|
||||||
RMAppEventType.ATTEMPT_KILLED,
|
RMAppEventType.ATTEMPT_KILLED,
|
||||||
"Application killed by user.");
|
"Application killed by user.", false);
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
case FAILED:
|
case FAILED:
|
||||||
|
@ -1004,10 +1022,17 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
||||||
// don't leave the tracking URL pointing to a non-existent AM
|
// don't leave the tracking URL pointing to a non-existent AM
|
||||||
appAttempt.setTrackingUrlToRMAppPage();
|
appAttempt.setTrackingUrlToRMAppPage();
|
||||||
appAttempt.invalidateAMHostAndPort();
|
appAttempt.invalidateAMHostAndPort();
|
||||||
|
if (appAttempt.submissionContext
|
||||||
|
.getKeepContainersAcrossApplicationAttempts()
|
||||||
|
&& !appAttempt.isLastAttempt
|
||||||
|
&& !appAttempt.submissionContext.getUnmanagedAM()) {
|
||||||
|
keepContainersAcrossAppAttempts = true;
|
||||||
|
}
|
||||||
appEvent =
|
appEvent =
|
||||||
new RMAppFailedAttemptEvent(applicationId,
|
new RMAppFailedAttemptEvent(applicationId,
|
||||||
RMAppEventType.ATTEMPT_FAILED,
|
RMAppEventType.ATTEMPT_FAILED, appAttempt.getDiagnostics(),
|
||||||
appAttempt.getDiagnostics());
|
keepContainersAcrossAppAttempts);
|
||||||
|
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
|
@ -1019,7 +1044,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
||||||
|
|
||||||
appAttempt.eventHandler.handle(appEvent);
|
appAttempt.eventHandler.handle(appEvent);
|
||||||
appAttempt.eventHandler.handle(new AppAttemptRemovedSchedulerEvent(
|
appAttempt.eventHandler.handle(new AppAttemptRemovedSchedulerEvent(
|
||||||
appAttemptId, finalAttemptState));
|
appAttemptId, finalAttemptState, keepContainersAcrossAppAttempts));
|
||||||
appAttempt.removeCredentials(appAttempt);
|
appAttempt.removeCredentials(appAttempt);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1045,6 +1070,11 @@ 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);
|
||||||
|
// TODO Today unmanaged AM client is waiting for app state to be Accepted to
|
||||||
|
// launch the AM. This is broken since we changed to start the attempt
|
||||||
|
// after the application is Accepted. We may need to introduce an attempt
|
||||||
|
// report that client can rely on to query the attempt state and choose to
|
||||||
|
// launch the unmanaged AM.
|
||||||
super.transition(appAttempt, event);
|
super.transition(appAttempt, event);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1346,6 +1376,20 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static final class ContainerFinishedAtFailedTransition
|
||||||
|
extends BaseTransition {
|
||||||
|
@Override
|
||||||
|
public void
|
||||||
|
transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) {
|
||||||
|
RMAppAttemptContainerFinishedEvent containerFinishedEvent =
|
||||||
|
(RMAppAttemptContainerFinishedEvent) event;
|
||||||
|
ContainerStatus containerStatus =
|
||||||
|
containerFinishedEvent.getContainerStatus();
|
||||||
|
// Normal container. Add it in completed containers list
|
||||||
|
appAttempt.justFinishedContainers.add(containerStatus);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private static class ContainerFinishedFinalStateSavedTransition extends
|
private static class ContainerFinishedFinalStateSavedTransition extends
|
||||||
BaseTransition {
|
BaseTransition {
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -0,0 +1,36 @@
|
||||||
|
/**
|
||||||
|
* 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.rmapp.attempt;
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
|
|
||||||
|
public class RMAppStartAttemptEvent extends RMAppAttemptEvent {
|
||||||
|
|
||||||
|
private final boolean transferStateFromPreviousAttempt;
|
||||||
|
|
||||||
|
public RMAppStartAttemptEvent(ApplicationAttemptId appAttemptId,
|
||||||
|
boolean transferStateFromPreviousAttempt) {
|
||||||
|
super(appAttemptId, RMAppAttemptEventType.START);
|
||||||
|
this.transferStateFromPreviousAttempt = transferStateFromPreviousAttempt;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean getTransferStateFromPreviousAttempt() {
|
||||||
|
return transferStateFromPreviousAttempt;
|
||||||
|
}
|
||||||
|
}
|
|
@ -61,7 +61,7 @@ public class AppSchedulingInfo {
|
||||||
new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
|
new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator());
|
||||||
final Map<Priority, Map<String, ResourceRequest>> requests =
|
final Map<Priority, Map<String, ResourceRequest>> requests =
|
||||||
new HashMap<Priority, Map<String, ResourceRequest>>();
|
new HashMap<Priority, Map<String, ResourceRequest>>();
|
||||||
final Set<String> blacklist = new HashSet<String>();
|
private Set<String> blacklist = new HashSet<String>();
|
||||||
|
|
||||||
//private final ApplicationStore store;
|
//private final ApplicationStore store;
|
||||||
private final ActiveUsersManager activeUsersManager;
|
private final ActiveUsersManager activeUsersManager;
|
||||||
|
@ -399,4 +399,15 @@ public class AppSchedulingInfo {
|
||||||
public synchronized void setQueue(Queue queue) {
|
public synchronized void setQueue(Queue queue) {
|
||||||
this.queue = queue;
|
this.queue = queue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public synchronized Set<String> getBlackList() {
|
||||||
|
return this.blacklist;
|
||||||
|
}
|
||||||
|
|
||||||
|
public synchronized void transferStateFromPreviousAppSchedulingInfo(
|
||||||
|
AppSchedulingInfo appInfo) {
|
||||||
|
// this.priorities = appInfo.getPriorities();
|
||||||
|
// this.requests = appInfo.getRequests();
|
||||||
|
this.blacklist = appInfo.getBlackList();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -26,6 +26,7 @@ public class SchedulerApplication {
|
||||||
|
|
||||||
private final Queue queue;
|
private final Queue queue;
|
||||||
private final String user;
|
private final String user;
|
||||||
|
private SchedulerApplicationAttempt currentAttempt;
|
||||||
|
|
||||||
public SchedulerApplication(Queue queue, String user) {
|
public SchedulerApplication(Queue queue, String user) {
|
||||||
this.queue = queue;
|
this.queue = queue;
|
||||||
|
@ -39,4 +40,12 @@ public class SchedulerApplication {
|
||||||
public String getUser() {
|
public String getUser() {
|
||||||
return user;
|
return user;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public SchedulerApplicationAttempt getCurrentAppAttempt() {
|
||||||
|
return currentAttempt;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setCurrentAppAttempt(SchedulerApplicationAttempt currentAttempt) {
|
||||||
|
this.currentAttempt = currentAttempt;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -64,7 +64,7 @@ public abstract class SchedulerApplicationAttempt {
|
||||||
|
|
||||||
protected final AppSchedulingInfo appSchedulingInfo;
|
protected final AppSchedulingInfo appSchedulingInfo;
|
||||||
|
|
||||||
protected final Map<ContainerId, RMContainer> liveContainers =
|
protected Map<ContainerId, RMContainer> liveContainers =
|
||||||
new HashMap<ContainerId, RMContainer>();
|
new HashMap<ContainerId, RMContainer>();
|
||||||
protected final Map<Priority, Map<NodeId, RMContainer>> reservedContainers =
|
protected final Map<Priority, Map<NodeId, RMContainer>> reservedContainers =
|
||||||
new HashMap<Priority, Map<NodeId, RMContainer>>();
|
new HashMap<Priority, Map<NodeId, RMContainer>>();
|
||||||
|
@ -73,7 +73,7 @@ public abstract class SchedulerApplicationAttempt {
|
||||||
|
|
||||||
protected final Resource currentReservation = Resource.newInstance(0, 0);
|
protected final Resource currentReservation = Resource.newInstance(0, 0);
|
||||||
private Resource resourceLimit = Resource.newInstance(0, 0);
|
private Resource resourceLimit = Resource.newInstance(0, 0);
|
||||||
protected final Resource currentConsumption = Resource.newInstance(0, 0);
|
protected Resource currentConsumption = Resource.newInstance(0, 0);
|
||||||
|
|
||||||
protected List<RMContainer> newlyAllocatedContainers =
|
protected List<RMContainer> newlyAllocatedContainers =
|
||||||
new ArrayList<RMContainer>();
|
new ArrayList<RMContainer>();
|
||||||
|
@ -407,4 +407,29 @@ public abstract class SchedulerApplicationAttempt {
|
||||||
Resources.add(currentConsumption, currentReservation));
|
Resources.add(currentConsumption, currentReservation));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public synchronized Map<ContainerId, RMContainer> getLiveContainersMap() {
|
||||||
|
return this.liveContainers;
|
||||||
|
}
|
||||||
|
|
||||||
|
public synchronized Resource getResourceLimit() {
|
||||||
|
return this.resourceLimit;
|
||||||
|
}
|
||||||
|
|
||||||
|
public synchronized Map<Priority, Long> getLastScheduledContainer() {
|
||||||
|
return this.lastScheduledContainer;
|
||||||
|
}
|
||||||
|
|
||||||
|
public synchronized void transferStateFromPreviousAttempt(
|
||||||
|
SchedulerApplicationAttempt appAttempt) {
|
||||||
|
this.liveContainers = appAttempt.getLiveContainersMap();
|
||||||
|
// this.reReservations = appAttempt.reReservations;
|
||||||
|
this.currentConsumption = appAttempt.getCurrentConsumption();
|
||||||
|
this.resourceLimit = appAttempt.getResourceLimit();
|
||||||
|
// this.currentReservation = appAttempt.currentReservation;
|
||||||
|
// this.newlyAllocatedContainers = appAttempt.newlyAllocatedContainers;
|
||||||
|
// this.schedulingOpportunities = appAttempt.schedulingOpportunities;
|
||||||
|
this.lastScheduledContainer = appAttempt.getLastScheduledContainer();
|
||||||
|
this.appSchedulingInfo
|
||||||
|
.transferStateFromPreviousAppSchedulingInfo(appAttempt.appSchedulingInfo);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,13 +19,13 @@
|
||||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
|
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Evolving;
|
import org.apache.hadoop.classification.InterfaceStability.Evolving;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||||
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.ApplicationResourceUsageReport;
|
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
|
||||||
|
@ -37,6 +37,7 @@ 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.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.event.EventHandler;
|
import org.apache.hadoop.yarn.event.EventHandler;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -170,4 +171,13 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
|
||||||
@LimitedPrivate("yarn")
|
@LimitedPrivate("yarn")
|
||||||
@Stable
|
@Stable
|
||||||
public List<ApplicationAttemptId> getAppsInQueue(String queueName);
|
public List<ApplicationAttemptId> getAppsInQueue(String queueName);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the container for the given containerId.
|
||||||
|
* @param containerId
|
||||||
|
* @return the container for the given containerId.
|
||||||
|
*/
|
||||||
|
@LimitedPrivate("yarn")
|
||||||
|
@Unstable
|
||||||
|
public RMContainer getRMContainer(ContainerId containerId);
|
||||||
}
|
}
|
||||||
|
|
|
@ -63,14 +63,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
|
||||||
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.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.rmnode.RMNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
|
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.SchedulerApplication;
|
|
||||||
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.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;
|
||||||
|
@ -194,10 +195,6 @@ public class CapacityScheduler
|
||||||
protected Map<ApplicationId, SchedulerApplication> applications =
|
protected Map<ApplicationId, SchedulerApplication> applications =
|
||||||
new ConcurrentHashMap<ApplicationId, SchedulerApplication>();
|
new ConcurrentHashMap<ApplicationId, SchedulerApplication>();
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
protected Map<ApplicationAttemptId, FiCaSchedulerApp> appAttempts =
|
|
||||||
new ConcurrentHashMap<ApplicationAttemptId, FiCaSchedulerApp>();
|
|
||||||
|
|
||||||
private boolean initialized = false;
|
private boolean initialized = false;
|
||||||
|
|
||||||
private ResourceCalculator calculator;
|
private ResourceCalculator calculator;
|
||||||
|
@ -464,21 +461,27 @@ public class CapacityScheduler
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized void addApplicationAttempt(
|
private synchronized void addApplicationAttempt(
|
||||||
ApplicationAttemptId applicationAttemptId) {
|
ApplicationAttemptId applicationAttemptId,
|
||||||
|
boolean transferStateFromPreviousAttempt) {
|
||||||
SchedulerApplication application =
|
SchedulerApplication application =
|
||||||
applications.get(applicationAttemptId.getApplicationId());
|
applications.get(applicationAttemptId.getApplicationId());
|
||||||
CSQueue queue = (CSQueue) application.getQueue();
|
CSQueue queue = (CSQueue) application.getQueue();
|
||||||
|
|
||||||
FiCaSchedulerApp SchedulerApp =
|
FiCaSchedulerApp attempt =
|
||||||
new FiCaSchedulerApp(applicationAttemptId, application.getUser(),
|
new FiCaSchedulerApp(applicationAttemptId, application.getUser(),
|
||||||
queue, queue.getActiveUsersManager(), rmContext);
|
queue, queue.getActiveUsersManager(), rmContext);
|
||||||
appAttempts.put(applicationAttemptId, SchedulerApp);
|
if (transferStateFromPreviousAttempt) {
|
||||||
queue.submitApplicationAttempt(SchedulerApp, application.getUser());
|
attempt.transferStateFromPreviousAttempt(application
|
||||||
|
.getCurrentAppAttempt());
|
||||||
|
}
|
||||||
|
application.setCurrentAppAttempt(attempt);
|
||||||
|
|
||||||
|
queue.submitApplicationAttempt(attempt, application.getUser());
|
||||||
LOG.info("Added Application Attempt " + applicationAttemptId
|
LOG.info("Added Application Attempt " + applicationAttemptId
|
||||||
+ " to scheduler from user " + application.getUser() + " in queue "
|
+ " to scheduler from user " + application.getUser() + " in queue "
|
||||||
+ queue.getQueueName());
|
+ queue.getQueueName());
|
||||||
rmContext.getDispatcher().getEventHandler().handle(
|
rmContext.getDispatcher().getEventHandler() .handle(
|
||||||
new RMAppAttemptEvent(applicationAttemptId,
|
new RMAppAttemptEvent(applicationAttemptId,
|
||||||
RMAppAttemptEventType.ATTEMPT_ADDED));
|
RMAppAttemptEventType.ATTEMPT_ADDED));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -486,7 +489,8 @@ public class CapacityScheduler
|
||||||
RMAppState finalState) {
|
RMAppState finalState) {
|
||||||
SchedulerApplication application = applications.get(applicationId);
|
SchedulerApplication application = applications.get(applicationId);
|
||||||
if (application == null){
|
if (application == null){
|
||||||
// The AppRemovedSchedulerEvent maybe sent on recovery for completed apps.
|
// The AppRemovedSchedulerEvent maybe sent on recovery for completed apps,
|
||||||
|
// ignore it.
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
CSQueue queue = (CSQueue) application.getQueue();
|
CSQueue queue = (CSQueue) application.getQueue();
|
||||||
|
@ -501,52 +505,56 @@ public class CapacityScheduler
|
||||||
|
|
||||||
private synchronized void doneApplicationAttempt(
|
private synchronized void doneApplicationAttempt(
|
||||||
ApplicationAttemptId applicationAttemptId,
|
ApplicationAttemptId applicationAttemptId,
|
||||||
RMAppAttemptState rmAppAttemptFinalState) {
|
RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
|
||||||
LOG.info("Application Attempt " + applicationAttemptId + " is done." +
|
LOG.info("Application Attempt " + applicationAttemptId + " is done." +
|
||||||
" finalState=" + rmAppAttemptFinalState);
|
" finalState=" + rmAppAttemptFinalState);
|
||||||
|
|
||||||
FiCaSchedulerApp application = getApplication(applicationAttemptId);
|
FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId);
|
||||||
|
SchedulerApplication application =
|
||||||
|
applications.get(applicationAttemptId.getApplicationId());
|
||||||
|
|
||||||
if (application == null) {
|
if (application == null || attempt == null) {
|
||||||
// throw new IOException("Unknown application " + applicationId +
|
|
||||||
// " has completed!");
|
|
||||||
LOG.info("Unknown application " + applicationAttemptId + " has completed!");
|
LOG.info("Unknown application " + applicationAttemptId + " has completed!");
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Release all the running containers
|
// Release all the allocated, acquired, running containers
|
||||||
for (RMContainer rmContainer : application.getLiveContainers()) {
|
for (RMContainer rmContainer : attempt.getLiveContainers()) {
|
||||||
completedContainer(rmContainer,
|
if (keepContainers
|
||||||
SchedulerUtils.createAbnormalContainerStatus(
|
&& rmContainer.getState().equals(RMContainerState.RUNNING)) {
|
||||||
rmContainer.getContainerId(),
|
// do not kill the running container in the case of work-preserving AM
|
||||||
SchedulerUtils.COMPLETED_APPLICATION),
|
// restart.
|
||||||
RMContainerEventType.KILL);
|
LOG.info("Skip killing " + rmContainer.getContainerId());
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
completedContainer(
|
||||||
|
rmContainer,
|
||||||
|
SchedulerUtils.createAbnormalContainerStatus(
|
||||||
|
rmContainer.getContainerId(), SchedulerUtils.COMPLETED_APPLICATION),
|
||||||
|
RMContainerEventType.KILL);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Release all reserved containers
|
// Release all reserved containers
|
||||||
for (RMContainer rmContainer : application.getReservedContainers()) {
|
for (RMContainer rmContainer : attempt.getReservedContainers()) {
|
||||||
completedContainer(rmContainer,
|
completedContainer(
|
||||||
SchedulerUtils.createAbnormalContainerStatus(
|
rmContainer,
|
||||||
rmContainer.getContainerId(),
|
SchedulerUtils.createAbnormalContainerStatus(
|
||||||
"Application Complete"),
|
rmContainer.getContainerId(), "Application Complete"),
|
||||||
RMContainerEventType.KILL);
|
RMContainerEventType.KILL);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Clean up pending requests, metrics etc.
|
// Clean up pending requests, metrics etc.
|
||||||
application.stop(rmAppAttemptFinalState);
|
attempt.stop(rmAppAttemptFinalState);
|
||||||
|
|
||||||
// Inform the queue
|
// Inform the queue
|
||||||
String queueName = application.getQueue().getQueueName();
|
String queueName = attempt.getQueue().getQueueName();
|
||||||
CSQueue queue = queues.get(queueName);
|
CSQueue queue = queues.get(queueName);
|
||||||
if (!(queue instanceof LeafQueue)) {
|
if (!(queue instanceof LeafQueue)) {
|
||||||
LOG.error("Cannot finish application " + "from non-leaf queue: "
|
LOG.error("Cannot finish application " + "from non-leaf queue: "
|
||||||
+ queueName);
|
+ queueName);
|
||||||
} else {
|
} else {
|
||||||
queue.finishApplicationAttempt(application, queue.getQueueName());
|
queue.finishApplicationAttempt(attempt, queue.getQueueName());
|
||||||
}
|
}
|
||||||
|
|
||||||
// Remove from our data-structure
|
|
||||||
appAttempts.remove(applicationAttemptId);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final Allocation EMPTY_ALLOCATION =
|
private static final Allocation EMPTY_ALLOCATION =
|
||||||
|
@ -558,7 +566,7 @@ public class CapacityScheduler
|
||||||
List<ResourceRequest> ask, List<ContainerId> release,
|
List<ResourceRequest> ask, List<ContainerId> release,
|
||||||
List<String> blacklistAdditions, List<String> blacklistRemovals) {
|
List<String> blacklistAdditions, List<String> blacklistRemovals) {
|
||||||
|
|
||||||
FiCaSchedulerApp application = getApplication(applicationAttemptId);
|
FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
|
||||||
if (application == null) {
|
if (application == null) {
|
||||||
LOG.info("Calling allocate on removed " +
|
LOG.info("Calling allocate on removed " +
|
||||||
"or non existant application " + applicationAttemptId);
|
"or non existant application " + applicationAttemptId);
|
||||||
|
@ -701,7 +709,7 @@ public class CapacityScheduler
|
||||||
RMContainer reservedContainer = node.getReservedContainer();
|
RMContainer reservedContainer = node.getReservedContainer();
|
||||||
if (reservedContainer != null) {
|
if (reservedContainer != null) {
|
||||||
FiCaSchedulerApp reservedApplication =
|
FiCaSchedulerApp reservedApplication =
|
||||||
getApplication(reservedContainer.getApplicationAttemptId());
|
getCurrentAttemptForContainer(reservedContainer.getContainerId());
|
||||||
|
|
||||||
// Try to fulfill the reservation
|
// Try to fulfill the reservation
|
||||||
LOG.info("Trying to fulfill reservation for application " +
|
LOG.info("Trying to fulfill reservation for application " +
|
||||||
|
@ -738,12 +746,11 @@ public class CapacityScheduler
|
||||||
|
|
||||||
private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) {
|
private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) {
|
||||||
// Get the application for the finished container
|
// Get the application for the finished container
|
||||||
ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId();
|
FiCaSchedulerApp application = getCurrentAttemptForContainer(containerId);
|
||||||
FiCaSchedulerApp application = getApplication(applicationAttemptId);
|
|
||||||
if (application == null) {
|
if (application == null) {
|
||||||
LOG.info("Unknown application: " + applicationAttemptId +
|
LOG.info("Unknown application "
|
||||||
" launched container " + containerId +
|
+ containerId.getApplicationAttemptId().getApplicationId()
|
||||||
" on node: " + node);
|
+ " launched container " + containerId + " on node: " + node);
|
||||||
this.rmContext.getDispatcher().getEventHandler()
|
this.rmContext.getDispatcher().getEventHandler()
|
||||||
.handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
|
.handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
|
||||||
return;
|
return;
|
||||||
|
@ -791,7 +798,8 @@ public class CapacityScheduler
|
||||||
{
|
{
|
||||||
AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
|
AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
|
||||||
(AppAttemptAddedSchedulerEvent) event;
|
(AppAttemptAddedSchedulerEvent) event;
|
||||||
addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId());
|
addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(),
|
||||||
|
appAttemptAddedEvent.getTransferStateFromPreviousAttempt());
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
case APP_ATTEMPT_REMOVED:
|
case APP_ATTEMPT_REMOVED:
|
||||||
|
@ -799,7 +807,8 @@ public class CapacityScheduler
|
||||||
AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
|
AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
|
||||||
(AppAttemptRemovedSchedulerEvent) event;
|
(AppAttemptRemovedSchedulerEvent) event;
|
||||||
doneApplicationAttempt(appAttemptRemovedEvent.getApplicationAttemptID(),
|
doneApplicationAttempt(appAttemptRemovedEvent.getApplicationAttemptID(),
|
||||||
appAttemptRemovedEvent.getFinalAttemptState());
|
appAttemptRemovedEvent.getFinalAttemptState(),
|
||||||
|
appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts());
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
case CONTAINER_EXPIRED:
|
case CONTAINER_EXPIRED:
|
||||||
|
@ -874,13 +883,13 @@ public class CapacityScheduler
|
||||||
Container container = rmContainer.getContainer();
|
Container container = rmContainer.getContainer();
|
||||||
|
|
||||||
// Get the application for the finished container
|
// Get the application for the finished container
|
||||||
ApplicationAttemptId applicationAttemptId =
|
FiCaSchedulerApp application =
|
||||||
container.getId().getApplicationAttemptId();
|
getCurrentAttemptForContainer(container.getId());
|
||||||
FiCaSchedulerApp application = getApplication(applicationAttemptId);
|
ApplicationId appId =
|
||||||
|
container.getId().getApplicationAttemptId().getApplicationId();
|
||||||
if (application == null) {
|
if (application == null) {
|
||||||
LOG.info("Container " + container + " of" +
|
LOG.info("Container " + container + " of" + " unknown application "
|
||||||
" unknown application " + applicationAttemptId +
|
+ appId + " completed with event " + event);
|
||||||
" completed with event " + event);
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -892,28 +901,33 @@ public class CapacityScheduler
|
||||||
queue.completedContainer(clusterResource, application, node,
|
queue.completedContainer(clusterResource, application, node,
|
||||||
rmContainer, containerStatus, event, null);
|
rmContainer, containerStatus, event, null);
|
||||||
|
|
||||||
LOG.info("Application " + applicationAttemptId +
|
LOG.info("Application attempt " + application.getApplicationAttemptId()
|
||||||
" released container " + container.getId() +
|
+ " released container " + container.getId() + " on node: " + node
|
||||||
" on node: " + node +
|
+ " with event: " + event);
|
||||||
" with event: " + event);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Lock(Lock.NoLock.class)
|
@Lock(Lock.NoLock.class)
|
||||||
FiCaSchedulerApp getApplication(ApplicationAttemptId applicationAttemptId) {
|
FiCaSchedulerApp getApplicationAttempt(
|
||||||
return appAttempts.get(applicationAttemptId);
|
ApplicationAttemptId applicationAttemptId) {
|
||||||
|
SchedulerApplication app =
|
||||||
|
applications.get(applicationAttemptId.getApplicationId());
|
||||||
|
if (app != null) {
|
||||||
|
return (FiCaSchedulerApp) app.getCurrentAppAttempt();
|
||||||
|
}
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SchedulerAppReport getSchedulerAppInfo(
|
public SchedulerAppReport getSchedulerAppInfo(
|
||||||
ApplicationAttemptId applicationAttemptId) {
|
ApplicationAttemptId applicationAttemptId) {
|
||||||
FiCaSchedulerApp app = getApplication(applicationAttemptId);
|
FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
|
||||||
return app == null ? null : new SchedulerAppReport(app);
|
return app == null ? null : new SchedulerAppReport(app);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ApplicationResourceUsageReport getAppResourceUsageReport(
|
public ApplicationResourceUsageReport getAppResourceUsageReport(
|
||||||
ApplicationAttemptId applicationAttemptId) {
|
ApplicationAttemptId applicationAttemptId) {
|
||||||
FiCaSchedulerApp app = getApplication(applicationAttemptId);
|
FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
|
||||||
return app == null ? null : app.getResourceUsageReport();
|
return app == null ? null : app.getResourceUsageReport();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -922,10 +936,22 @@ public class CapacityScheduler
|
||||||
return nodes.get(nodeId);
|
return nodes.get(nodeId);
|
||||||
}
|
}
|
||||||
|
|
||||||
private RMContainer getRMContainer(ContainerId containerId) {
|
@Override
|
||||||
FiCaSchedulerApp application =
|
public RMContainer getRMContainer(ContainerId containerId) {
|
||||||
getApplication(containerId.getApplicationAttemptId());
|
FiCaSchedulerApp attempt = getCurrentAttemptForContainer(containerId);
|
||||||
return (application == null) ? null : application.getRMContainer(containerId);
|
return (attempt == null) ? null : attempt.getRMContainer(containerId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public FiCaSchedulerApp getCurrentAttemptForContainer(
|
||||||
|
ContainerId containerId) {
|
||||||
|
SchedulerApplication app =
|
||||||
|
applications.get(containerId.getApplicationAttemptId()
|
||||||
|
.getApplicationId());
|
||||||
|
if (app != null) {
|
||||||
|
return (FiCaSchedulerApp) app.getCurrentAppAttempt();
|
||||||
|
}
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -958,7 +984,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 = appAttempts.get(aid);
|
FiCaSchedulerApp app = getApplicationAttempt(aid);
|
||||||
if (app != null) {
|
if (app != null) {
|
||||||
app.addPreemptContainer(cont.getContainerId());
|
app.addPreemptContainer(cont.getContainerId());
|
||||||
}
|
}
|
||||||
|
|
|
@ -219,7 +219,8 @@ public class FiCaSchedulerNode extends SchedulerNode {
|
||||||
" on node " + this.reservedContainer.getReservedNode());
|
" on node " + this.reservedContainer.getReservedNode());
|
||||||
}
|
}
|
||||||
|
|
||||||
// Cannot reserve more than one application on a given node!
|
// Cannot reserve more than one application attempt on a given node!
|
||||||
|
// Reservation is still against attempt.
|
||||||
if (!this.reservedContainer.getContainer().getId().getApplicationAttemptId().equals(
|
if (!this.reservedContainer.getContainer().getId().getApplicationAttemptId().equals(
|
||||||
reservedContainer.getContainer().getId().getApplicationAttemptId())) {
|
reservedContainer.getContainer().getId().getApplicationAttemptId())) {
|
||||||
throw new IllegalStateException("Trying to reserve" +
|
throw new IllegalStateException("Trying to reserve" +
|
||||||
|
|
|
@ -23,14 +23,21 @@ 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 boolean transferStateFromPreviousAttempt;
|
||||||
|
|
||||||
public AppAttemptAddedSchedulerEvent(
|
public AppAttemptAddedSchedulerEvent(
|
||||||
ApplicationAttemptId applicationAttemptId) {
|
ApplicationAttemptId applicationAttemptId,
|
||||||
|
boolean transferStateFromPreviousAttempt) {
|
||||||
super(SchedulerEventType.APP_ATTEMPT_ADDED);
|
super(SchedulerEventType.APP_ATTEMPT_ADDED);
|
||||||
this.applicationAttemptId = applicationAttemptId;
|
this.applicationAttemptId = applicationAttemptId;
|
||||||
|
this.transferStateFromPreviousAttempt = transferStateFromPreviousAttempt;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ApplicationAttemptId getApplicationAttemptId() {
|
public ApplicationAttemptId getApplicationAttemptId() {
|
||||||
return applicationAttemptId;
|
return applicationAttemptId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean getTransferStateFromPreviousAttempt() {
|
||||||
|
return transferStateFromPreviousAttempt;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,13 +25,15 @@ public class AppAttemptRemovedSchedulerEvent extends SchedulerEvent {
|
||||||
|
|
||||||
private final ApplicationAttemptId applicationAttemptId;
|
private final ApplicationAttemptId applicationAttemptId;
|
||||||
private final RMAppAttemptState finalAttemptState;
|
private final RMAppAttemptState finalAttemptState;
|
||||||
|
private final boolean keepContainersAcrossAppAttempts;
|
||||||
|
|
||||||
public AppAttemptRemovedSchedulerEvent(
|
public AppAttemptRemovedSchedulerEvent(
|
||||||
ApplicationAttemptId applicationAttemptId,
|
ApplicationAttemptId applicationAttemptId,
|
||||||
RMAppAttemptState finalAttemptState) {
|
RMAppAttemptState finalAttemptState, boolean keepContainers) {
|
||||||
super(SchedulerEventType.APP_ATTEMPT_REMOVED);
|
super(SchedulerEventType.APP_ATTEMPT_REMOVED);
|
||||||
this.applicationAttemptId = applicationAttemptId;
|
this.applicationAttemptId = applicationAttemptId;
|
||||||
this.finalAttemptState = finalAttemptState;
|
this.finalAttemptState = finalAttemptState;
|
||||||
|
this.keepContainersAcrossAppAttempts = keepContainers;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ApplicationAttemptId getApplicationAttemptID() {
|
public ApplicationAttemptId getApplicationAttemptID() {
|
||||||
|
@ -41,4 +43,8 @@ public class AppAttemptRemovedSchedulerEvent extends SchedulerEvent {
|
||||||
public RMAppAttemptState getFinalAttemptState() {
|
public RMAppAttemptState getFinalAttemptState() {
|
||||||
return this.finalAttemptState;
|
return this.finalAttemptState;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean getKeepContainersAcrossAppAttempts() {
|
||||||
|
return this.keepContainersAcrossAppAttempts;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -162,12 +162,6 @@ public class FairScheduler implements ResourceScheduler {
|
||||||
protected Map<ApplicationId, SchedulerApplication> applications =
|
protected Map<ApplicationId, SchedulerApplication> applications =
|
||||||
new ConcurrentHashMap<ApplicationId, SchedulerApplication>();
|
new ConcurrentHashMap<ApplicationId, SchedulerApplication>();
|
||||||
|
|
||||||
// This stores per-application-attempt scheduling information, indexed by
|
|
||||||
// attempt ID's for fast lookup.
|
|
||||||
@VisibleForTesting
|
|
||||||
protected Map<ApplicationAttemptId, FSSchedulerApp> appAttempts =
|
|
||||||
new ConcurrentHashMap<ApplicationAttemptId, FSSchedulerApp>();
|
|
||||||
|
|
||||||
// Nodes in the cluster, indexed by NodeId
|
// Nodes in the cluster, indexed by NodeId
|
||||||
private Map<NodeId, FSSchedulerNode> nodes =
|
private Map<NodeId, FSSchedulerNode> nodes =
|
||||||
new ConcurrentHashMap<NodeId, FSSchedulerNode>();
|
new ConcurrentHashMap<NodeId, FSSchedulerNode>();
|
||||||
|
@ -262,10 +256,21 @@ public class FairScheduler implements ResourceScheduler {
|
||||||
return queueMgr;
|
return queueMgr;
|
||||||
}
|
}
|
||||||
|
|
||||||
private RMContainer getRMContainer(ContainerId containerId) {
|
@Override
|
||||||
FSSchedulerApp application =
|
public RMContainer getRMContainer(ContainerId containerId) {
|
||||||
appAttempts.get(containerId.getApplicationAttemptId());
|
FSSchedulerApp attempt = getCurrentAttemptForContainer(containerId);
|
||||||
return (application == null) ? null : application.getRMContainer(containerId);
|
return (attempt == null) ? null : attempt.getRMContainer(containerId);
|
||||||
|
}
|
||||||
|
|
||||||
|
private FSSchedulerApp getCurrentAttemptForContainer(
|
||||||
|
ContainerId containerId) {
|
||||||
|
SchedulerApplication app =
|
||||||
|
applications.get(containerId.getApplicationAttemptId()
|
||||||
|
.getApplicationId());
|
||||||
|
if (app != null) {
|
||||||
|
return (FSSchedulerApp) app.getCurrentAppAttempt();
|
||||||
|
}
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -640,7 +645,8 @@ public class FairScheduler implements ResourceScheduler {
|
||||||
applications.put(applicationId, application);
|
applications.put(applicationId, application);
|
||||||
|
|
||||||
LOG.info("Accepted application " + applicationId + " from user: " + user
|
LOG.info("Accepted application " + applicationId + " from user: " + user
|
||||||
+ ", in queue: " + queueName);
|
+ ", in queue: " + queueName + ", currently num of applications: "
|
||||||
|
+ applications.size());
|
||||||
rmContext.getDispatcher().getEventHandler()
|
rmContext.getDispatcher().getEventHandler()
|
||||||
.handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
|
.handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
|
||||||
}
|
}
|
||||||
|
@ -649,31 +655,35 @@ public class FairScheduler implements ResourceScheduler {
|
||||||
* Add a new application attempt to the scheduler.
|
* Add a new application attempt to the scheduler.
|
||||||
*/
|
*/
|
||||||
protected synchronized void addApplicationAttempt(
|
protected synchronized void addApplicationAttempt(
|
||||||
ApplicationAttemptId applicationAttemptId) {
|
ApplicationAttemptId applicationAttemptId,
|
||||||
|
boolean transferStateFromPreviousAttempt) {
|
||||||
SchedulerApplication application =
|
SchedulerApplication application =
|
||||||
applications.get(applicationAttemptId.getApplicationId());
|
applications.get(applicationAttemptId.getApplicationId());
|
||||||
String user = application.getUser();
|
String user = application.getUser();
|
||||||
FSLeafQueue queue = (FSLeafQueue) application.getQueue();
|
FSLeafQueue queue = (FSLeafQueue) application.getQueue();
|
||||||
|
|
||||||
FSSchedulerApp schedulerApp =
|
FSSchedulerApp attempt =
|
||||||
new FSSchedulerApp(applicationAttemptId, user,
|
new FSSchedulerApp(applicationAttemptId, user,
|
||||||
queue, new ActiveUsersManager(getRootQueueMetrics()),
|
queue, new ActiveUsersManager(getRootQueueMetrics()),
|
||||||
rmContext);
|
rmContext);
|
||||||
|
if (transferStateFromPreviousAttempt) {
|
||||||
|
attempt.transferStateFromPreviousAttempt(application
|
||||||
|
.getCurrentAppAttempt());
|
||||||
|
}
|
||||||
|
application.setCurrentAppAttempt(attempt);
|
||||||
|
|
||||||
boolean runnable = maxRunningEnforcer.canAppBeRunnable(queue, user);
|
boolean runnable = maxRunningEnforcer.canAppBeRunnable(queue, user);
|
||||||
queue.addApp(schedulerApp, runnable);
|
queue.addApp(attempt, runnable);
|
||||||
if (runnable) {
|
if (runnable) {
|
||||||
maxRunningEnforcer.trackRunnableApp(schedulerApp);
|
maxRunningEnforcer.trackRunnableApp(attempt);
|
||||||
} else {
|
} else {
|
||||||
maxRunningEnforcer.trackNonRunnableApp(schedulerApp);
|
maxRunningEnforcer.trackNonRunnableApp(attempt);
|
||||||
}
|
}
|
||||||
|
|
||||||
queue.getMetrics().submitApp(user, applicationAttemptId.getAttemptId());
|
queue.getMetrics().submitApp(user, applicationAttemptId.getAttemptId());
|
||||||
appAttempts.put(applicationAttemptId, schedulerApp);
|
|
||||||
|
|
||||||
LOG.info("Added Application Attempt " + applicationAttemptId
|
LOG.info("Added Application Attempt " + applicationAttemptId
|
||||||
+ " to scheduler from user: " + user + ", currently active: "
|
+ " to scheduler from user: " + user);
|
||||||
+ appAttempts.size());
|
|
||||||
rmContext.getDispatcher().getEventHandler().handle(
|
rmContext.getDispatcher().getEventHandler().handle(
|
||||||
new RMAppAttemptEvent(applicationAttemptId,
|
new RMAppAttemptEvent(applicationAttemptId,
|
||||||
RMAppAttemptEventType.ATTEMPT_ADDED));
|
RMAppAttemptEventType.ATTEMPT_ADDED));
|
||||||
|
@ -709,19 +719,27 @@ public class FairScheduler implements ResourceScheduler {
|
||||||
|
|
||||||
private synchronized void removeApplicationAttempt(
|
private synchronized void removeApplicationAttempt(
|
||||||
ApplicationAttemptId applicationAttemptId,
|
ApplicationAttemptId applicationAttemptId,
|
||||||
RMAppAttemptState rmAppAttemptFinalState) {
|
RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
|
||||||
LOG.info("Application " + applicationAttemptId + " is done." +
|
LOG.info("Application " + applicationAttemptId + " is done." +
|
||||||
" finalState=" + rmAppAttemptFinalState);
|
" finalState=" + rmAppAttemptFinalState);
|
||||||
|
SchedulerApplication application =
|
||||||
|
applications.get(applicationAttemptId.getApplicationId());
|
||||||
|
FSSchedulerApp attempt = getSchedulerApp(applicationAttemptId);
|
||||||
|
|
||||||
FSSchedulerApp application = appAttempts.get(applicationAttemptId);
|
if (attempt == null || application == null) {
|
||||||
|
|
||||||
if (application == null) {
|
|
||||||
LOG.info("Unknown application " + applicationAttemptId + " has completed!");
|
LOG.info("Unknown application " + applicationAttemptId + " has completed!");
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Release all the running containers
|
// Release all the running containers
|
||||||
for (RMContainer rmContainer : application.getLiveContainers()) {
|
for (RMContainer rmContainer : attempt.getLiveContainers()) {
|
||||||
|
if (keepContainers
|
||||||
|
&& rmContainer.getState().equals(RMContainerState.RUNNING)) {
|
||||||
|
// do not kill the running container in the case of work-preserving AM
|
||||||
|
// restart.
|
||||||
|
LOG.info("Skip killing " + rmContainer.getContainerId());
|
||||||
|
continue;
|
||||||
|
}
|
||||||
completedContainer(rmContainer,
|
completedContainer(rmContainer,
|
||||||
SchedulerUtils.createAbnormalContainerStatus(
|
SchedulerUtils.createAbnormalContainerStatus(
|
||||||
rmContainer.getContainerId(),
|
rmContainer.getContainerId(),
|
||||||
|
@ -730,30 +748,26 @@ public class FairScheduler implements ResourceScheduler {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Release all reserved containers
|
// Release all reserved containers
|
||||||
for (RMContainer rmContainer : application.getReservedContainers()) {
|
for (RMContainer rmContainer : attempt.getReservedContainers()) {
|
||||||
completedContainer(rmContainer,
|
completedContainer(rmContainer,
|
||||||
SchedulerUtils.createAbnormalContainerStatus(
|
SchedulerUtils.createAbnormalContainerStatus(
|
||||||
rmContainer.getContainerId(),
|
rmContainer.getContainerId(),
|
||||||
"Application Complete"),
|
"Application Complete"),
|
||||||
RMContainerEventType.KILL);
|
RMContainerEventType.KILL);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Clean up pending requests, metrics etc.
|
// Clean up pending requests, metrics etc.
|
||||||
application.stop(rmAppAttemptFinalState);
|
attempt.stop(rmAppAttemptFinalState);
|
||||||
|
|
||||||
// Inform the queue
|
// Inform the queue
|
||||||
FSLeafQueue queue = queueMgr.getLeafQueue(application.getQueue()
|
FSLeafQueue queue = queueMgr.getLeafQueue(attempt.getQueue()
|
||||||
.getQueueName(), false);
|
.getQueueName(), false);
|
||||||
boolean wasRunnable = queue.removeApp(application);
|
boolean wasRunnable = queue.removeApp(attempt);
|
||||||
|
|
||||||
if (wasRunnable) {
|
if (wasRunnable) {
|
||||||
maxRunningEnforcer.updateRunnabilityOnAppRemoval(application);
|
maxRunningEnforcer.updateRunnabilityOnAppRemoval(attempt);
|
||||||
} else {
|
} else {
|
||||||
maxRunningEnforcer.untrackNonRunnableApp(application);
|
maxRunningEnforcer.untrackNonRunnableApp(attempt);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Remove from our data-structure
|
|
||||||
appAttempts.remove(applicationAttemptId);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -769,11 +783,13 @@ public class FairScheduler implements ResourceScheduler {
|
||||||
Container container = rmContainer.getContainer();
|
Container container = rmContainer.getContainer();
|
||||||
|
|
||||||
// Get the application for the finished container
|
// Get the application for the finished container
|
||||||
ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId();
|
FSSchedulerApp application =
|
||||||
FSSchedulerApp application = appAttempts.get(applicationAttemptId);
|
getCurrentAttemptForContainer(container.getId());
|
||||||
|
ApplicationId appId =
|
||||||
|
container.getId().getApplicationAttemptId().getApplicationId();
|
||||||
if (application == null) {
|
if (application == null) {
|
||||||
LOG.info("Container " + container + " of" +
|
LOG.info("Container " + container + " of" +
|
||||||
" unknown application " + applicationAttemptId +
|
" unknown application attempt " + appId +
|
||||||
" completed with event " + event);
|
" completed with event " + event);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -790,10 +806,9 @@ public class FairScheduler implements ResourceScheduler {
|
||||||
updateRootQueueMetrics();
|
updateRootQueueMetrics();
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG.info("Application " + applicationAttemptId +
|
LOG.info("Application attempt " + application.getApplicationAttemptId()
|
||||||
" released container " + container.getId() +
|
+ " released container " + container.getId() + " on node: " + node
|
||||||
" on node: " + node +
|
+ " with event: " + event);
|
||||||
" with event: " + event);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized void addNode(RMNode node) {
|
private synchronized void addNode(RMNode node) {
|
||||||
|
@ -844,7 +859,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 = appAttempts.get(appAttemptId);
|
FSSchedulerApp application = getSchedulerApp(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);
|
||||||
|
@ -914,12 +929,11 @@ 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();
|
FSSchedulerApp application = getCurrentAttemptForContainer(containerId);
|
||||||
FSSchedulerApp application = appAttempts.get(applicationAttemptId);
|
|
||||||
if (application == null) {
|
if (application == null) {
|
||||||
LOG.info("Unknown application: " + applicationAttemptId +
|
LOG.info("Unknown application "
|
||||||
" launched container " + containerId +
|
+ containerId.getApplicationAttemptId().getApplicationId()
|
||||||
" on node: " + node);
|
+ " launched container " + containerId + " on node: " + node);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1058,28 +1072,34 @@ public class FairScheduler implements ResourceScheduler {
|
||||||
}
|
}
|
||||||
|
|
||||||
public FSSchedulerApp getSchedulerApp(ApplicationAttemptId appAttemptId) {
|
public FSSchedulerApp getSchedulerApp(ApplicationAttemptId appAttemptId) {
|
||||||
return appAttempts.get(appAttemptId);
|
SchedulerApplication app =
|
||||||
|
applications.get(appAttemptId.getApplicationId());
|
||||||
|
if (app != null) {
|
||||||
|
return (FSSchedulerApp) app.getCurrentAppAttempt();
|
||||||
|
}
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SchedulerAppReport getSchedulerAppInfo(
|
public SchedulerAppReport getSchedulerAppInfo(
|
||||||
ApplicationAttemptId appAttemptId) {
|
ApplicationAttemptId appAttemptId) {
|
||||||
if (!appAttempts.containsKey(appAttemptId)) {
|
FSSchedulerApp attempt = getSchedulerApp(appAttemptId);
|
||||||
|
if (attempt == null) {
|
||||||
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(appAttempts.get(appAttemptId));
|
return new SchedulerAppReport(attempt);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ApplicationResourceUsageReport getAppResourceUsageReport(
|
public ApplicationResourceUsageReport getAppResourceUsageReport(
|
||||||
ApplicationAttemptId appAttemptId) {
|
ApplicationAttemptId appAttemptId) {
|
||||||
FSSchedulerApp app = appAttempts.get(appAttemptId);
|
FSSchedulerApp attempt = getSchedulerApp(appAttemptId);
|
||||||
if (app == null) {
|
if (attempt == null) {
|
||||||
LOG.error("Request for appInfo of unknown attempt" + appAttemptId);
|
LOG.error("Request for appInfo of unknown attempt" + appAttemptId);
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
return app.getResourceUsageReport();
|
return attempt.getResourceUsageReport();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1145,7 +1165,8 @@ public class FairScheduler implements ResourceScheduler {
|
||||||
}
|
}
|
||||||
AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
|
AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
|
||||||
(AppAttemptAddedSchedulerEvent) event;
|
(AppAttemptAddedSchedulerEvent) event;
|
||||||
addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId());
|
addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(),
|
||||||
|
appAttemptAddedEvent.getTransferStateFromPreviousAttempt());
|
||||||
break;
|
break;
|
||||||
case APP_ATTEMPT_REMOVED:
|
case APP_ATTEMPT_REMOVED:
|
||||||
if (!(event instanceof AppAttemptRemovedSchedulerEvent)) {
|
if (!(event instanceof AppAttemptRemovedSchedulerEvent)) {
|
||||||
|
@ -1153,8 +1174,10 @@ public class FairScheduler implements ResourceScheduler {
|
||||||
}
|
}
|
||||||
AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
|
AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
|
||||||
(AppAttemptRemovedSchedulerEvent) event;
|
(AppAttemptRemovedSchedulerEvent) event;
|
||||||
removeApplicationAttempt(appAttemptRemovedEvent.getApplicationAttemptID(),
|
removeApplicationAttempt(
|
||||||
appAttemptRemovedEvent.getFinalAttemptState());
|
appAttemptRemovedEvent.getApplicationAttemptID(),
|
||||||
|
appAttemptRemovedEvent.getFinalAttemptState(),
|
||||||
|
appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts());
|
||||||
break;
|
break;
|
||||||
case CONTAINER_EXPIRED:
|
case CONTAINER_EXPIRED:
|
||||||
if (!(event instanceof ContainerExpiredSchedulerEvent)) {
|
if (!(event instanceof ContainerExpiredSchedulerEvent)) {
|
||||||
|
|
|
@ -67,6 +67,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
|
||||||
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.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.rmnode.RMNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
|
||||||
|
@ -123,15 +124,11 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
private Resource maximumAllocation;
|
private Resource maximumAllocation;
|
||||||
private boolean usePortForNodeName;
|
private boolean usePortForNodeName;
|
||||||
|
|
||||||
|
// Use ConcurrentSkipListMap because applications need to be ordered
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
protected Map<ApplicationId, SchedulerApplication> applications =
|
protected Map<ApplicationId, SchedulerApplication> applications =
|
||||||
new ConcurrentSkipListMap<ApplicationId, SchedulerApplication>();
|
new ConcurrentSkipListMap<ApplicationId, SchedulerApplication>();
|
||||||
|
|
||||||
// Use ConcurrentSkipListMap because applications need to be ordered
|
|
||||||
@VisibleForTesting
|
|
||||||
protected Map<ApplicationAttemptId, FiCaSchedulerApp> appAttempts
|
|
||||||
= new ConcurrentSkipListMap<ApplicationAttemptId, FiCaSchedulerApp>();
|
|
||||||
|
|
||||||
private ActiveUsersManager activeUsersManager;
|
private ActiveUsersManager activeUsersManager;
|
||||||
|
|
||||||
private static final String DEFAULT_QUEUE_NAME = "default";
|
private static final String DEFAULT_QUEUE_NAME = "default";
|
||||||
|
@ -270,7 +267,7 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
public Allocation allocate(
|
public Allocation allocate(
|
||||||
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
|
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
|
||||||
List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) {
|
List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) {
|
||||||
FiCaSchedulerApp application = getApplication(applicationAttemptId);
|
FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
|
||||||
if (application == null) {
|
if (application == null) {
|
||||||
LOG.error("Calling allocate on removed " +
|
LOG.error("Calling allocate on removed " +
|
||||||
"or non existant application " + applicationAttemptId);
|
"or non existant application " + applicationAttemptId);
|
||||||
|
@ -336,22 +333,26 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
FiCaSchedulerApp getApplication(
|
FiCaSchedulerApp getApplicationAttempt(ApplicationAttemptId applicationAttemptId) {
|
||||||
ApplicationAttemptId applicationAttemptId) {
|
SchedulerApplication app =
|
||||||
return appAttempts.get(applicationAttemptId);
|
applications.get(applicationAttemptId.getApplicationId());
|
||||||
|
if (app != null) {
|
||||||
|
return (FiCaSchedulerApp) app.getCurrentAppAttempt();
|
||||||
|
}
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SchedulerAppReport getSchedulerAppInfo(
|
public SchedulerAppReport getSchedulerAppInfo(
|
||||||
ApplicationAttemptId applicationAttemptId) {
|
ApplicationAttemptId applicationAttemptId) {
|
||||||
FiCaSchedulerApp app = getApplication(applicationAttemptId);
|
FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
|
||||||
return app == null ? null : new SchedulerAppReport(app);
|
return app == null ? null : new SchedulerAppReport(app);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ApplicationResourceUsageReport getAppResourceUsageReport(
|
public ApplicationResourceUsageReport getAppResourceUsageReport(
|
||||||
ApplicationAttemptId applicationAttemptId) {
|
ApplicationAttemptId applicationAttemptId) {
|
||||||
FiCaSchedulerApp app = getApplication(applicationAttemptId);
|
FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId);
|
||||||
return app == null ? null : app.getResourceUsageReport();
|
return app == null ? null : app.getResourceUsageReport();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -364,13 +365,15 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
SchedulerApplication application =
|
SchedulerApplication application =
|
||||||
new SchedulerApplication(null, user);
|
new SchedulerApplication(null, user);
|
||||||
applications.put(applicationId, application);
|
applications.put(applicationId, application);
|
||||||
LOG.info("Accepted application " + applicationId + " from user: " + user);
|
LOG.info("Accepted application " + applicationId + " from user: " + user
|
||||||
|
+ ", currently num of applications: " + applications.size());
|
||||||
rmContext.getDispatcher().getEventHandler()
|
rmContext.getDispatcher().getEventHandler()
|
||||||
.handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
|
.handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized void addApplicationAttempt(
|
private synchronized void
|
||||||
ApplicationAttemptId appAttemptId) {
|
addApplicationAttempt(ApplicationAttemptId appAttemptId,
|
||||||
|
boolean transferStateFromPreviousAttempt) {
|
||||||
SchedulerApplication application =
|
SchedulerApplication application =
|
||||||
applications.get(appAttemptId.getApplicationId());
|
applications.get(appAttemptId.getApplicationId());
|
||||||
String user = application.getUser();
|
String user = application.getUser();
|
||||||
|
@ -378,11 +381,16 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
FiCaSchedulerApp schedulerApp =
|
FiCaSchedulerApp schedulerApp =
|
||||||
new FiCaSchedulerApp(appAttemptId, user, DEFAULT_QUEUE,
|
new FiCaSchedulerApp(appAttemptId, user, DEFAULT_QUEUE,
|
||||||
activeUsersManager, this.rmContext);
|
activeUsersManager, this.rmContext);
|
||||||
appAttempts.put(appAttemptId, schedulerApp);
|
|
||||||
|
if (transferStateFromPreviousAttempt) {
|
||||||
|
schedulerApp.transferStateFromPreviousAttempt(application
|
||||||
|
.getCurrentAppAttempt());
|
||||||
|
}
|
||||||
|
application.setCurrentAppAttempt(schedulerApp);
|
||||||
|
|
||||||
metrics.submitApp(user, appAttemptId.getAttemptId());
|
metrics.submitApp(user, appAttemptId.getAttemptId());
|
||||||
LOG.info("Added Application Attempt " + appAttemptId
|
LOG.info("Added Application Attempt " + appAttemptId
|
||||||
+ " to scheduler from user " + application.getUser()
|
+ " 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.ATTEMPT_ADDED));
|
RMAppAttemptEventType.ATTEMPT_ADDED));
|
||||||
|
@ -400,28 +408,33 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
|
|
||||||
private synchronized void doneApplicationAttempt(
|
private synchronized void doneApplicationAttempt(
|
||||||
ApplicationAttemptId applicationAttemptId,
|
ApplicationAttemptId applicationAttemptId,
|
||||||
RMAppAttemptState rmAppAttemptFinalState)
|
RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
FiCaSchedulerApp application = getApplication(applicationAttemptId);
|
FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId);
|
||||||
if (application == null) {
|
SchedulerApplication application =
|
||||||
|
applications.get(applicationAttemptId.getApplicationId());
|
||||||
|
if (application == null || attempt == null) {
|
||||||
throw new IOException("Unknown application " + applicationAttemptId +
|
throw new IOException("Unknown application " + applicationAttemptId +
|
||||||
" has completed!");
|
" has completed!");
|
||||||
}
|
}
|
||||||
|
|
||||||
// Kill all 'live' containers
|
// Kill all 'live' containers
|
||||||
for (RMContainer container : application.getLiveContainers()) {
|
for (RMContainer container : attempt.getLiveContainers()) {
|
||||||
|
if (keepContainers
|
||||||
|
&& container.getState().equals(RMContainerState.RUNNING)) {
|
||||||
|
// do not kill the running container in the case of work-preserving AM
|
||||||
|
// restart.
|
||||||
|
LOG.info("Skip killing " + container.getContainerId());
|
||||||
|
continue;
|
||||||
|
}
|
||||||
containerCompleted(container,
|
containerCompleted(container,
|
||||||
SchedulerUtils.createAbnormalContainerStatus(
|
SchedulerUtils.createAbnormalContainerStatus(
|
||||||
container.getContainerId(),
|
container.getContainerId(), SchedulerUtils.COMPLETED_APPLICATION),
|
||||||
SchedulerUtils.COMPLETED_APPLICATION),
|
RMContainerEventType.KILL);
|
||||||
RMContainerEventType.KILL);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Clean up pending requests, metrics etc.
|
// Clean up pending requests, metrics etc.
|
||||||
application.stop(rmAppAttemptFinalState);
|
attempt.stop(rmAppAttemptFinalState);
|
||||||
|
|
||||||
// Remove the application
|
|
||||||
appAttempts.remove(applicationAttemptId);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -432,12 +445,13 @@ 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=" + appAttempts.size());
|
" #applications=" + applications.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 : appAttempts
|
for (Map.Entry<ApplicationId, SchedulerApplication> e : applications
|
||||||
.entrySet()) {
|
.entrySet()) {
|
||||||
FiCaSchedulerApp application = e.getValue();
|
FiCaSchedulerApp application =
|
||||||
|
(FiCaSchedulerApp) e.getValue().getCurrentAppAttempt();
|
||||||
LOG.debug("pre-assignContainers");
|
LOG.debug("pre-assignContainers");
|
||||||
application.showRequests();
|
application.showRequests();
|
||||||
synchronized (application) {
|
synchronized (application) {
|
||||||
|
@ -474,8 +488,10 @@ 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 : appAttempts.values()) {
|
for (SchedulerApplication application : applications.values()) {
|
||||||
application.setHeadroom(Resources.subtract(clusterResource, usedResource));
|
FiCaSchedulerApp attempt =
|
||||||
|
(FiCaSchedulerApp) application.getCurrentAppAttempt();
|
||||||
|
attempt.setHeadroom(Resources.subtract(clusterResource, usedResource));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -744,7 +760,8 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
{
|
{
|
||||||
AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
|
AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
|
||||||
(AppAttemptAddedSchedulerEvent) event;
|
(AppAttemptAddedSchedulerEvent) event;
|
||||||
addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId());
|
addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(),
|
||||||
|
appAttemptAddedEvent.getTransferStateFromPreviousAttempt());
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
case APP_ATTEMPT_REMOVED:
|
case APP_ATTEMPT_REMOVED:
|
||||||
|
@ -754,7 +771,8 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
try {
|
try {
|
||||||
doneApplicationAttempt(
|
doneApplicationAttempt(
|
||||||
appAttemptRemovedEvent.getApplicationAttemptID(),
|
appAttemptRemovedEvent.getApplicationAttemptID(),
|
||||||
appAttemptRemovedEvent.getFinalAttemptState());
|
appAttemptRemovedEvent.getFinalAttemptState(),
|
||||||
|
appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts());
|
||||||
} catch(IOException ie) {
|
} catch(IOException ie) {
|
||||||
LOG.error("Unable to remove application "
|
LOG.error("Unable to remove application "
|
||||||
+ appAttemptRemovedEvent.getApplicationAttemptID(), ie);
|
+ appAttemptRemovedEvent.getApplicationAttemptID(), ie);
|
||||||
|
@ -780,12 +798,11 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
|
|
||||||
private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) {
|
private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) {
|
||||||
// Get the application for the finished container
|
// Get the application for the finished container
|
||||||
ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId();
|
FiCaSchedulerApp application = getCurrentAttemptForContainer(containerId);
|
||||||
FiCaSchedulerApp application = getApplication(applicationAttemptId);
|
|
||||||
if (application == null) {
|
if (application == null) {
|
||||||
LOG.info("Unknown application: " + applicationAttemptId +
|
LOG.info("Unknown application "
|
||||||
" launched container " + containerId +
|
+ containerId.getApplicationAttemptId().getApplicationId()
|
||||||
" on node: " + node);
|
+ " launched container " + containerId + " on node: " + node);
|
||||||
// Some unknown container sneaked into the system. Kill it.
|
// Some unknown container sneaked into the system. Kill it.
|
||||||
this.rmContext.getDispatcher().getEventHandler()
|
this.rmContext.getDispatcher().getEventHandler()
|
||||||
.handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
|
.handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId));
|
||||||
|
@ -806,14 +823,16 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
|
|
||||||
// Get the application for the finished container
|
// Get the application for the finished container
|
||||||
Container container = rmContainer.getContainer();
|
Container container = rmContainer.getContainer();
|
||||||
ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId();
|
FiCaSchedulerApp application =
|
||||||
FiCaSchedulerApp application = getApplication(applicationAttemptId);
|
getCurrentAttemptForContainer(container.getId());
|
||||||
|
ApplicationId appId =
|
||||||
|
container.getId().getApplicationAttemptId().getApplicationId();
|
||||||
|
|
||||||
// Get the node on which the container was allocated
|
// Get the node on which the container was allocated
|
||||||
FiCaSchedulerNode node = getNode(container.getNodeId());
|
FiCaSchedulerNode node = getNode(container.getNodeId());
|
||||||
|
|
||||||
if (application == null) {
|
if (application == null) {
|
||||||
LOG.info("Unknown application: " + applicationAttemptId +
|
LOG.info("Unknown application: " + appId +
|
||||||
" released container " + container.getId() +
|
" released container " + container.getId() +
|
||||||
" on node: " + node +
|
" on node: " + node +
|
||||||
" with event: " + event);
|
" with event: " + event);
|
||||||
|
@ -829,7 +848,7 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
// Update total usage
|
// Update total usage
|
||||||
Resources.subtractFrom(usedResource, container.getResource());
|
Resources.subtractFrom(usedResource, container.getResource());
|
||||||
|
|
||||||
LOG.info("Application " + applicationAttemptId +
|
LOG.info("Application attempt " + application.getApplicationAttemptId() +
|
||||||
" released container " + container.getId() +
|
" released container " + container.getId() +
|
||||||
" on node: " + node +
|
" on node: " + node +
|
||||||
" with event: " + event);
|
" with event: " + event);
|
||||||
|
@ -888,10 +907,21 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
return node == null ? null : new SchedulerNodeReport(node);
|
return node == null ? null : new SchedulerNodeReport(node);
|
||||||
}
|
}
|
||||||
|
|
||||||
private RMContainer getRMContainer(ContainerId containerId) {
|
@Override
|
||||||
FiCaSchedulerApp application =
|
public RMContainer getRMContainer(ContainerId containerId) {
|
||||||
getApplication(containerId.getApplicationAttemptId());
|
FiCaSchedulerApp attempt = getCurrentAttemptForContainer(containerId);
|
||||||
return (application == null) ? null : application.getRMContainer(containerId);
|
return (attempt == null) ? null : attempt.getRMContainer(containerId);
|
||||||
|
}
|
||||||
|
|
||||||
|
private FiCaSchedulerApp getCurrentAttemptForContainer(
|
||||||
|
ContainerId containerId) {
|
||||||
|
SchedulerApplication app =
|
||||||
|
applications.get(containerId.getApplicationAttemptId()
|
||||||
|
.getApplicationId());
|
||||||
|
if (app != null) {
|
||||||
|
return (FiCaSchedulerApp) app.getCurrentAppAttempt();
|
||||||
|
}
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -908,12 +938,12 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
||||||
@Override
|
@Override
|
||||||
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> attempts = new ArrayList<ApplicationAttemptId>(
|
||||||
appAttempts.size());
|
applications.size());
|
||||||
for (FiCaSchedulerApp app : appAttempts.values()) {
|
for (SchedulerApplication app : applications.values()) {
|
||||||
apps.add(app.getApplicationAttemptId());
|
attempts.add(app.getCurrentAppAttempt().getApplicationAttemptId());
|
||||||
}
|
}
|
||||||
return apps;
|
return attempts;
|
||||||
} else {
|
} else {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
|
@ -171,7 +171,7 @@ public class Application {
|
||||||
new AppAddedSchedulerEvent(this.applicationId, this.queue, "user");
|
new AppAddedSchedulerEvent(this.applicationId, this.queue, "user");
|
||||||
scheduler.handle(addAppEvent);
|
scheduler.handle(addAppEvent);
|
||||||
AppAttemptAddedSchedulerEvent addAttemptEvent =
|
AppAttemptAddedSchedulerEvent addAttemptEvent =
|
||||||
new AppAttemptAddedSchedulerEvent(this.applicationAttemptId);
|
new AppAttemptAddedSchedulerEvent(this.applicationAttemptId, false);
|
||||||
scheduler.handle(addAttemptEvent);
|
scheduler.handle(addAttemptEvent);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
||||||
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.YarnVersionInfo;
|
import org.apache.hadoop.yarn.util.YarnVersionInfo;
|
||||||
|
import org.mortbay.log.Log;
|
||||||
|
|
||||||
public class MockNM {
|
public class MockNM {
|
||||||
|
|
||||||
|
@ -130,12 +131,13 @@ public class MockNM {
|
||||||
int containerId, ContainerState containerState) throws Exception {
|
int containerId, ContainerState containerState) throws Exception {
|
||||||
HashMap<ApplicationId, List<ContainerStatus>> nodeUpdate =
|
HashMap<ApplicationId, List<ContainerStatus>> nodeUpdate =
|
||||||
new HashMap<ApplicationId, List<ContainerStatus>>(1);
|
new HashMap<ApplicationId, List<ContainerStatus>>(1);
|
||||||
ContainerStatus amContainerStatus = BuilderUtils.newContainerStatus(
|
ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
|
||||||
BuilderUtils.newContainerId(attemptId, 1),
|
BuilderUtils.newContainerId(attemptId, containerId), containerState,
|
||||||
ContainerState.COMPLETE, "Success", 0);
|
"Success", 0);
|
||||||
ArrayList<ContainerStatus> containerStatusList =
|
ArrayList<ContainerStatus> containerStatusList =
|
||||||
new ArrayList<ContainerStatus>(1);
|
new ArrayList<ContainerStatus>(1);
|
||||||
containerStatusList.add(amContainerStatus);
|
containerStatusList.add(containerStatus);
|
||||||
|
Log.info("ContainerStatus: " + containerStatus);
|
||||||
nodeUpdate.put(attemptId.getApplicationId(), containerStatusList);
|
nodeUpdate.put(attemptId.getApplicationId(), containerStatusList);
|
||||||
return nodeHeartbeat(nodeUpdate, true);
|
return nodeHeartbeat(nodeUpdate, true);
|
||||||
}
|
}
|
||||||
|
@ -152,6 +154,7 @@ public class MockNM {
|
||||||
status.setResponseId(resId);
|
status.setResponseId(resId);
|
||||||
status.setNodeId(nodeId);
|
status.setNodeId(nodeId);
|
||||||
for (Map.Entry<ApplicationId, List<ContainerStatus>> entry : conts.entrySet()) {
|
for (Map.Entry<ApplicationId, List<ContainerStatus>> entry : conts.entrySet()) {
|
||||||
|
Log.info("entry.getValue() " + entry.getValue());
|
||||||
status.setContainersStatuses(entry.getValue());
|
status.setContainersStatuses(entry.getValue());
|
||||||
}
|
}
|
||||||
NodeHealthStatus healthStatus = Records.newRecord(NodeHealthStatus.class);
|
NodeHealthStatus healthStatus = Records.newRecord(NodeHealthStatus.class);
|
||||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.io.DataOutputBuffer;
|
||||||
import org.apache.hadoop.security.Credentials;
|
import org.apache.hadoop.security.Credentials;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
|
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
|
||||||
|
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
|
||||||
|
@ -40,7 +41,10 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
||||||
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.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
||||||
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||||
import org.apache.hadoop.yarn.api.records.NodeState;
|
import org.apache.hadoop.yarn.api.records.NodeState;
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
|
@ -56,6 +60,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
|
||||||
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.RMAppAttemptLaunchFailedEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
||||||
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.RMNodeEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
|
||||||
|
@ -122,6 +128,33 @@ public class MockRM extends ResourceManager {
|
||||||
attempt.getAppAttemptState());
|
attempt.getAppAttemptState());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void waitForContainerAllocated(MockNM nm, ContainerId containerId)
|
||||||
|
throws Exception {
|
||||||
|
int timeoutSecs = 0;
|
||||||
|
while (getResourceScheduler().getRMContainer(containerId) == null
|
||||||
|
&& timeoutSecs++ < 40) {
|
||||||
|
System.out.println("Waiting for" + containerId + " to be allocated.");
|
||||||
|
nm.nodeHeartbeat(true);
|
||||||
|
Thread.sleep(200);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void waitForState(MockNM nm, ContainerId containerId,
|
||||||
|
RMContainerState containerState) throws Exception {
|
||||||
|
RMContainer container = getResourceScheduler().getRMContainer(containerId);
|
||||||
|
Assert.assertNotNull("Container shouldn't be null", container);
|
||||||
|
int timeoutSecs = 0;
|
||||||
|
while (!containerState.equals(container.getState()) && timeoutSecs++ < 40) {
|
||||||
|
System.out.println("Container : " + containerId + " State is : "
|
||||||
|
+ container.getState() + " Waiting for state : " + containerState);
|
||||||
|
nm.nodeHeartbeat(true);
|
||||||
|
Thread.sleep(300);
|
||||||
|
}
|
||||||
|
System.out.println("Container State is : " + container.getState());
|
||||||
|
Assert.assertEquals("Container state is not correct (timedout)",
|
||||||
|
containerState, container.getState());
|
||||||
|
}
|
||||||
|
|
||||||
// get new application id
|
// get new application id
|
||||||
public GetNewApplicationResponse getNewAppId() throws Exception {
|
public GetNewApplicationResponse getNewAppId() throws Exception {
|
||||||
ApplicationClientProtocol client = getClientRMService();
|
ApplicationClientProtocol client = getClientRMService();
|
||||||
|
@ -172,7 +205,17 @@ public class MockRM extends ResourceManager {
|
||||||
public RMApp submitApp(int masterMemory, String name, String user,
|
public RMApp submitApp(int masterMemory, String name, String user,
|
||||||
Map<ApplicationAccessType, String> acls, boolean unmanaged, String queue,
|
Map<ApplicationAccessType, String> acls, boolean unmanaged, String queue,
|
||||||
int maxAppAttempts, Credentials ts, String appType,
|
int maxAppAttempts, Credentials ts, String appType,
|
||||||
boolean waitForAccepted) throws Exception {
|
boolean waitForAccepted)
|
||||||
|
throws Exception {
|
||||||
|
return submitApp(masterMemory, name, user, acls, unmanaged, queue,
|
||||||
|
maxAppAttempts, ts, appType, waitForAccepted, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
public RMApp submitApp(int masterMemory, String name, String user,
|
||||||
|
Map<ApplicationAccessType, String> acls, boolean unmanaged, String queue,
|
||||||
|
int maxAppAttempts, Credentials ts, String appType,
|
||||||
|
boolean waitForAccepted, boolean keepContainers)
|
||||||
|
throws Exception {
|
||||||
ApplicationClientProtocol client = getClientRMService();
|
ApplicationClientProtocol client = getClientRMService();
|
||||||
GetNewApplicationResponse resp = client.getNewApplication(Records
|
GetNewApplicationResponse resp = client.getNewApplication(Records
|
||||||
.newRecord(GetNewApplicationRequest.class));
|
.newRecord(GetNewApplicationRequest.class));
|
||||||
|
@ -182,6 +225,7 @@ public class MockRM extends ResourceManager {
|
||||||
.newRecord(SubmitApplicationRequest.class);
|
.newRecord(SubmitApplicationRequest.class);
|
||||||
ApplicationSubmissionContext sub = Records
|
ApplicationSubmissionContext sub = Records
|
||||||
.newRecord(ApplicationSubmissionContext.class);
|
.newRecord(ApplicationSubmissionContext.class);
|
||||||
|
sub.setKeepContainersAcrossApplicationAttempts(keepContainers);
|
||||||
sub.setApplicationId(appId);
|
sub.setApplicationId(appId);
|
||||||
sub.setApplicationName(name);
|
sub.setApplicationName(name);
|
||||||
sub.setMaxAppAttempts(maxAppAttempts);
|
sub.setMaxAppAttempts(maxAppAttempts);
|
||||||
|
@ -421,4 +465,26 @@ public class MockRM extends ResourceManager {
|
||||||
// override to disable webapp
|
// override to disable webapp
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void finishApplicationMaster(RMApp rmApp, MockRM rm, MockNM nm,
|
||||||
|
MockAM am) throws Exception {
|
||||||
|
FinishApplicationMasterRequest req =
|
||||||
|
FinishApplicationMasterRequest.newInstance(
|
||||||
|
FinalApplicationStatus.SUCCEEDED, "", "");
|
||||||
|
am.unregisterAppAttempt(req);
|
||||||
|
am.waitForState(RMAppAttemptState.FINISHING);
|
||||||
|
nm.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
|
||||||
|
am.waitForState(RMAppAttemptState.FINISHED);
|
||||||
|
rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
|
||||||
|
throws Exception {
|
||||||
|
RMAppAttempt attempt = app.getCurrentAppAttempt();
|
||||||
|
nm.nodeHeartbeat(true);
|
||||||
|
MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());
|
||||||
|
am.registerAppAttempt();
|
||||||
|
rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
|
||||||
|
return am;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
rmContext, yarnScheduler, null, asContext, config, false);
|
||||||
when(app.getCurrentAppAttempt()).thenReturn(rmAppAttemptImpl);
|
when(app.getCurrentAppAttempt()).thenReturn(rmAppAttemptImpl);
|
||||||
return app;
|
return app;
|
||||||
}
|
}
|
||||||
|
|
|
@ -302,7 +302,7 @@ public class TestFifoScheduler {
|
||||||
new AppAddedSchedulerEvent(appId1, "queue", "user");
|
new AppAddedSchedulerEvent(appId1, "queue", "user");
|
||||||
fs.handle(appEvent);
|
fs.handle(appEvent);
|
||||||
SchedulerEvent attemptEvent =
|
SchedulerEvent attemptEvent =
|
||||||
new AppAttemptAddedSchedulerEvent(appAttemptId1);
|
new AppAttemptAddedSchedulerEvent(appAttemptId1, false);
|
||||||
fs.handle(attemptEvent);
|
fs.handle(attemptEvent);
|
||||||
|
|
||||||
List<ContainerId> emptyId = new ArrayList<ContainerId>();
|
List<ContainerId> emptyId = new ArrayList<ContainerId>();
|
||||||
|
@ -396,7 +396,7 @@ public class TestFifoScheduler {
|
||||||
new AppAddedSchedulerEvent(appId1, "queue", "user");
|
new AppAddedSchedulerEvent(appId1, "queue", "user");
|
||||||
fs.handle(appEvent);
|
fs.handle(appEvent);
|
||||||
SchedulerEvent attemptEvent =
|
SchedulerEvent attemptEvent =
|
||||||
new AppAttemptAddedSchedulerEvent(appAttemptId1);
|
new AppAttemptAddedSchedulerEvent(appAttemptId1, false);
|
||||||
fs.handle(attemptEvent);
|
fs.handle(attemptEvent);
|
||||||
|
|
||||||
ApplicationId appId2 = BuilderUtils.newApplicationId(200, 2);
|
ApplicationId appId2 = BuilderUtils.newApplicationId(200, 2);
|
||||||
|
@ -406,7 +406,7 @@ public class TestFifoScheduler {
|
||||||
new AppAddedSchedulerEvent(appId2, "queue", "user");
|
new AppAddedSchedulerEvent(appId2, "queue", "user");
|
||||||
fs.handle(appEvent2);
|
fs.handle(appEvent2);
|
||||||
SchedulerEvent attemptEvent2 =
|
SchedulerEvent attemptEvent2 =
|
||||||
new AppAttemptAddedSchedulerEvent(appAttemptId2);
|
new AppAttemptAddedSchedulerEvent(appAttemptId2, false);
|
||||||
fs.handle(attemptEvent2);
|
fs.handle(attemptEvent2);
|
||||||
|
|
||||||
List<ContainerId> emptyId = new ArrayList<ContainerId>();
|
List<ContainerId> emptyId = new ArrayList<ContainerId>();
|
||||||
|
|
|
@ -28,7 +28,6 @@ import junit.framework.Assert;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
|
||||||
|
@ -38,7 +37,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
||||||
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.ContainerState;
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
||||||
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
||||||
import org.apache.hadoop.yarn.api.records.NMToken;
|
import org.apache.hadoop.yarn.api.records.NMToken;
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
import org.apache.hadoop.yarn.api.records.Token;
|
import org.apache.hadoop.yarn.api.records.Token;
|
||||||
|
@ -295,6 +293,8 @@ public class TestRM {
|
||||||
nm2.nodeHeartbeat(attempt.getAppAttemptId(), container.getId().getId(),
|
nm2.nodeHeartbeat(attempt.getAppAttemptId(), container.getId().getId(),
|
||||||
ContainerState.COMPLETE);
|
ContainerState.COMPLETE);
|
||||||
}
|
}
|
||||||
|
nm1.nodeHeartbeat(am.getApplicationAttemptId(), 1,
|
||||||
|
ContainerState.COMPLETE);
|
||||||
am.waitForState(RMAppAttemptState.FINISHED);
|
am.waitForState(RMAppAttemptState.FINISHED);
|
||||||
Assert.assertFalse(nmTokenSecretManager
|
Assert.assertFalse(nmTokenSecretManager
|
||||||
.isApplicationAttemptRegistered(attempt.getAppAttemptId()));
|
.isApplicationAttemptRegistered(attempt.getAppAttemptId()));
|
||||||
|
@ -389,19 +389,19 @@ public class TestRM {
|
||||||
MockNM nm1 =
|
MockNM nm1 =
|
||||||
new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
|
new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
|
||||||
nm1.registerNode();
|
nm1.registerNode();
|
||||||
MockAM am1 = launchAM(app1, rm1, nm1);
|
MockAM am1 = MockRM.launchAM(app1, rm1, nm1);
|
||||||
finishApplicationMaster(app1, rm1, nm1, am1);
|
MockRM.finishApplicationMaster(app1, rm1, nm1, am1);
|
||||||
|
|
||||||
// a failed app
|
// a failed app
|
||||||
RMApp app2 = rm1.submitApp(200);
|
RMApp app2 = rm1.submitApp(200);
|
||||||
MockAM am2 = launchAM(app2, rm1, nm1);
|
MockAM am2 = MockRM.launchAM(app2, rm1, nm1);
|
||||||
nm1.nodeHeartbeat(am2.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
|
nm1.nodeHeartbeat(am2.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
|
||||||
am2.waitForState(RMAppAttemptState.FAILED);
|
am2.waitForState(RMAppAttemptState.FAILED);
|
||||||
rm1.waitForState(app2.getApplicationId(), RMAppState.FAILED);
|
rm1.waitForState(app2.getApplicationId(), RMAppState.FAILED);
|
||||||
|
|
||||||
// a killed app
|
// a killed app
|
||||||
RMApp app3 = rm1.submitApp(200);
|
RMApp app3 = rm1.submitApp(200);
|
||||||
MockAM am3 = launchAM(app3, rm1, nm1);
|
MockAM am3 = MockRM.launchAM(app3, rm1, nm1);
|
||||||
rm1.killApp(app3.getApplicationId());
|
rm1.killApp(app3.getApplicationId());
|
||||||
rm1.waitForState(app3.getApplicationId(), RMAppState.KILLED);
|
rm1.waitForState(app3.getApplicationId(), RMAppState.KILLED);
|
||||||
rm1.waitForState(am3.getApplicationAttemptId(), RMAppAttemptState.KILLED);
|
rm1.waitForState(am3.getApplicationAttemptId(), RMAppAttemptState.KILLED);
|
||||||
|
@ -441,7 +441,7 @@ public class TestRM {
|
||||||
|
|
||||||
// a failed app
|
// a failed app
|
||||||
RMApp app2 = rm1.submitApp(200);
|
RMApp app2 = rm1.submitApp(200);
|
||||||
MockAM am2 = launchAM(app2, rm1, nm1);
|
MockAM am2 = MockRM.launchAM(app2, rm1, nm1);
|
||||||
nm1
|
nm1
|
||||||
.nodeHeartbeat(am2.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
|
.nodeHeartbeat(am2.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
|
||||||
am2.waitForState(RMAppAttemptState.FAILED);
|
am2.waitForState(RMAppAttemptState.FAILED);
|
||||||
|
@ -458,28 +458,6 @@ public class TestRM {
|
||||||
Assert.assertEquals(-1, report1.getRpcPort());
|
Assert.assertEquals(-1, report1.getRpcPort());
|
||||||
}
|
}
|
||||||
|
|
||||||
private MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
|
|
||||||
throws Exception {
|
|
||||||
RMAppAttempt attempt = app.getCurrentAppAttempt();
|
|
||||||
nm.nodeHeartbeat(true);
|
|
||||||
MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());
|
|
||||||
am.registerAppAttempt();
|
|
||||||
rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
|
|
||||||
return am;
|
|
||||||
}
|
|
||||||
|
|
||||||
private void finishApplicationMaster(RMApp rmApp, MockRM rm, MockNM nm,
|
|
||||||
MockAM am) throws Exception {
|
|
||||||
FinishApplicationMasterRequest req =
|
|
||||||
FinishApplicationMasterRequest.newInstance(
|
|
||||||
FinalApplicationStatus.SUCCEEDED, "", "");
|
|
||||||
am.unregisterAppAttempt(req);
|
|
||||||
am.waitForState(RMAppAttemptState.FINISHING);
|
|
||||||
nm.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
|
|
||||||
am.waitForState(RMAppAttemptState.FINISHED);
|
|
||||||
rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
public static void main(String[] args) throws Exception {
|
||||||
TestRM t = new TestRM();
|
TestRM t = new TestRM();
|
||||||
t.testGetNewAppId();
|
t.testGetNewAppId();
|
||||||
|
|
|
@ -164,7 +164,7 @@ public class TestResourceManager {
|
||||||
// Notify scheduler application is finished.
|
// Notify scheduler application is finished.
|
||||||
AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
|
AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
|
||||||
new AppAttemptRemovedSchedulerEvent(
|
new AppAttemptRemovedSchedulerEvent(
|
||||||
application.getApplicationAttemptId(), RMAppAttemptState.FINISHED);
|
application.getApplicationAttemptId(), RMAppAttemptState.FINISHED, false);
|
||||||
resourceManager.getResourceScheduler().handle(appRemovedEvent1);
|
resourceManager.getResourceScheduler().handle(appRemovedEvent1);
|
||||||
|
|
||||||
checkResourceUsage(nm1, nm2);
|
checkResourceUsage(nm1, nm2);
|
||||||
|
|
|
@ -18,49 +18,30 @@
|
||||||
|
|
||||||
package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager;
|
package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.util.ArrayList;
|
||||||
import java.nio.ByteBuffer;
|
import java.util.HashMap;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
|
||||||
|
|
||||||
import junit.framework.Assert;
|
import junit.framework.Assert;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
|
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
||||||
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
||||||
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.NodeId;
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
||||||
import org.apache.hadoop.yarn.api.records.Priority;
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.QueueInfo;
|
|
||||||
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
|
||||||
import org.apache.hadoop.yarn.api.records.Resource;
|
|
||||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||||
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.server.resourcemanager.MockAM;
|
||||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
||||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
|
||||||
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.YarnScheduler;
|
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
|
|
||||||
import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
|
|
||||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
||||||
import org.junit.After;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -68,238 +49,164 @@ import org.junit.Test;
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
public class TestAMRestart {
|
public class TestAMRestart {
|
||||||
// private static final Log LOG = LogFactory.getLog(TestAMRestart.class);
|
|
||||||
// ApplicationsManagerImpl appImpl;
|
@Test
|
||||||
// RMContext asmContext = new RMContextImpl(new MemStore());
|
public void testAMRestartWithExistingContainers() throws Exception {
|
||||||
// ApplicationTokenSecretManager appTokenSecretManager =
|
YarnConfiguration conf = new YarnConfiguration();
|
||||||
// new ApplicationTokenSecretManager();
|
conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
|
||||||
// DummyResourceScheduler scheduler;
|
|
||||||
// private ClientRMService clientRMService;
|
MockRM rm1 = new MockRM(conf);
|
||||||
// int count = 0;
|
rm1.start();
|
||||||
// ApplicationId appID;
|
RMApp app1 =
|
||||||
// final int maxFailures = 3;
|
rm1.submitApp(200, "name", "user",
|
||||||
// AtomicInteger launchNotify = new AtomicInteger();
|
new HashMap<ApplicationAccessType, String>(), false, "default", -1,
|
||||||
// AtomicInteger schedulerNotify = new AtomicInteger();
|
null, "MAPREDUCE", false, true);
|
||||||
// volatile boolean stop = false;
|
MockNM nm1 =
|
||||||
// int schedulerAddApplication = 0;
|
new MockNM("127.0.0.1:1234", 10240, rm1.getResourceTrackerService());
|
||||||
// int schedulerRemoveApplication = 0;
|
nm1.registerNode();
|
||||||
// int launcherLaunchCalled = 0;
|
MockNM nm2 =
|
||||||
// int launcherCleanupCalled = 0;
|
new MockNM("127.0.0.1:2351", 4089, rm1.getResourceTrackerService());
|
||||||
// private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
|
nm2.registerNode();
|
||||||
//
|
|
||||||
// private class ExtApplicationsManagerImpl extends ApplicationsManagerImpl {
|
MockAM am1 = MockRM.launchAM(app1, rm1, nm1);
|
||||||
// public ExtApplicationsManagerImpl(
|
int NUM_CONTAINERS = 3;
|
||||||
// ApplicationTokenSecretManager applicationTokenSecretManager,
|
// allocate NUM_CONTAINERS containers
|
||||||
// YarnScheduler scheduler, RMContext asmContext) {
|
am1.allocate("127.0.0.1", 1024, NUM_CONTAINERS,
|
||||||
// super(applicationTokenSecretManager, scheduler, asmContext);
|
new ArrayList<ContainerId>());
|
||||||
// }
|
nm1.nodeHeartbeat(true);
|
||||||
//
|
|
||||||
// @Override
|
// wait for containers to be allocated.
|
||||||
// public EventHandler<ASMEvent<AMLauncherEventType>> createNewApplicationMasterLauncher(
|
List<Container> containers =
|
||||||
// ApplicationTokenSecretManager tokenSecretManager) {
|
am1.allocate(new ArrayList<ResourceRequest>(),
|
||||||
// return new DummyAMLauncher();
|
new ArrayList<ContainerId>()).getAllocatedContainers();
|
||||||
// }
|
while (containers.size() != NUM_CONTAINERS) {
|
||||||
// }
|
nm1.nodeHeartbeat(true);
|
||||||
//
|
containers.addAll(am1.allocate(new ArrayList<ResourceRequest>(),
|
||||||
// private class DummyAMLauncher implements EventHandler<ASMEvent<AMLauncherEventType>> {
|
new ArrayList<ContainerId>()).getAllocatedContainers());
|
||||||
//
|
Thread.sleep(200);
|
||||||
// public DummyAMLauncher() {
|
}
|
||||||
// asmContext.getDispatcher().register(AMLauncherEventType.class, this);
|
|
||||||
// new Thread() {
|
// launch the 2nd container, for testing running container transferred.
|
||||||
// public void run() {
|
nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.RUNNING);
|
||||||
// while (!stop) {
|
ContainerId containerId2 =
|
||||||
// LOG.info("DEBUG -- waiting for launch");
|
ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
|
||||||
// synchronized(launchNotify) {
|
rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
|
||||||
// while (launchNotify.get() == 0) {
|
|
||||||
// try {
|
// launch the 3rd container, for testing container allocated by previous
|
||||||
// launchNotify.wait();
|
// attempt is completed by the next new attempt/
|
||||||
// } catch (InterruptedException e) {
|
nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.RUNNING);
|
||||||
// }
|
ContainerId containerId3 =
|
||||||
// }
|
ContainerId.newInstance(am1.getApplicationAttemptId(), 3);
|
||||||
// asmContext.getDispatcher().getEventHandler().handle(
|
rm1.waitForState(nm1, containerId3, RMContainerState.RUNNING);
|
||||||
// new ApplicationEvent(
|
|
||||||
// ApplicationEventType.LAUNCHED, appID));
|
// 4th container still in AQUIRED state. for testing Acquired container is
|
||||||
// launchNotify.addAndGet(-1);
|
// always killed.
|
||||||
// }
|
ContainerId containerId4 =
|
||||||
// }
|
ContainerId.newInstance(am1.getApplicationAttemptId(), 4);
|
||||||
// }
|
rm1.waitForState(nm1, containerId4, RMContainerState.ACQUIRED);
|
||||||
// }.start();
|
|
||||||
// }
|
// 5th container is in Allocated state. for testing allocated container is
|
||||||
//
|
// always killed.
|
||||||
// @Override
|
am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
|
||||||
// public void handle(ASMEvent<AMLauncherEventType> event) {
|
nm1.nodeHeartbeat(true);
|
||||||
// switch (event.getType()) {
|
ContainerId containerId5 =
|
||||||
// case CLEANUP:
|
ContainerId.newInstance(am1.getApplicationAttemptId(), 5);
|
||||||
// launcherCleanupCalled++;
|
rm1.waitForContainerAllocated(nm1, containerId5);
|
||||||
// break;
|
rm1.waitForState(nm1, containerId5, RMContainerState.ALLOCATED);
|
||||||
// case LAUNCH:
|
|
||||||
// LOG.info("DEBUG -- launching");
|
// 6th container is in Reserved state.
|
||||||
// launcherLaunchCalled++;
|
am1.allocate("127.0.0.1", 6000, 1, new ArrayList<ContainerId>());
|
||||||
// synchronized (launchNotify) {
|
ContainerId containerId6 =
|
||||||
// launchNotify.addAndGet(1);
|
ContainerId.newInstance(am1.getApplicationAttemptId(), 6);
|
||||||
// launchNotify.notify();
|
nm1.nodeHeartbeat(true);
|
||||||
// }
|
SchedulerApplicationAttempt schedulerAttempt =
|
||||||
// break;
|
((CapacityScheduler) rm1.getResourceScheduler())
|
||||||
// default:
|
.getCurrentAttemptForContainer(containerId6);
|
||||||
// break;
|
while (schedulerAttempt.getReservedContainers().size() == 0) {
|
||||||
// }
|
System.out.println("Waiting for container " + containerId6
|
||||||
// }
|
+ " to be reserved.");
|
||||||
// }
|
nm1.nodeHeartbeat(true);
|
||||||
//
|
Thread.sleep(200);
|
||||||
// private class DummyResourceScheduler implements ResourceScheduler {
|
}
|
||||||
//
|
// assert containerId6 is reserved.
|
||||||
// @Override
|
Assert.assertEquals(containerId6, schedulerAttempt.getReservedContainers()
|
||||||
// public void removeNode(RMNode node) {
|
.get(0).getContainerId());
|
||||||
// }
|
|
||||||
//
|
// fail the AM by sending CONTAINER_FINISHED event without registering.
|
||||||
// @Override
|
nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1, ContainerState.COMPLETE);
|
||||||
// public Allocation allocate(ApplicationId applicationId,
|
am1.waitForState(RMAppAttemptState.FAILED);
|
||||||
// List<ResourceRequest> ask, List<Container> release) throws IOException {
|
|
||||||
// Container container = recordFactory.newRecordInstance(Container.class);
|
// wait for some time. previous AM's running containers should still remain
|
||||||
// container.setContainerToken(recordFactory.newRecordInstance(ContainerToken.class));
|
// in scheduler even though am failed
|
||||||
// container.setNodeId(recordFactory.newRecordInstance(NodeId.class));
|
Thread.sleep(3000);
|
||||||
// container.setContainerManagerAddress("localhost");
|
rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
|
||||||
// container.setNodeHttpAddress("localhost:8042");
|
// acquired/allocated containers are cleaned up.
|
||||||
// container.setId(recordFactory.newRecordInstance(ContainerId.class));
|
Assert.assertNull(rm1.getResourceScheduler().getRMContainer(containerId4));
|
||||||
// container.getId().setAppId(appID);
|
Assert.assertNull(rm1.getResourceScheduler().getRMContainer(containerId5));
|
||||||
// container.getId().setId(count);
|
|
||||||
// count++;
|
// wait for app to start a new attempt.
|
||||||
// return new Allocation(Arrays.asList(container), Resources.none());
|
rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
|
||||||
// }
|
// assert this is a new AM.
|
||||||
//
|
ApplicationAttemptId newAttemptId =
|
||||||
// @Override
|
app1.getCurrentAppAttempt().getAppAttemptId();
|
||||||
// public void handle(ASMEvent<ApplicationTrackerEventType> event) {
|
Assert.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId()));
|
||||||
// switch (event.getType()) {
|
MockAM am2 = MockRM.launchAM(app1, rm1, nm1);
|
||||||
// case ADD:
|
|
||||||
// schedulerAddApplication++;
|
// complete container by sending the container complete event which has earlier
|
||||||
// break;
|
// attempt's attemptId
|
||||||
// case EXPIRE:
|
nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.COMPLETE);
|
||||||
// schedulerRemoveApplication++;
|
rm1.waitForState(nm1, containerId3, RMContainerState.COMPLETED);
|
||||||
// LOG.info("REMOVING app : " + schedulerRemoveApplication);
|
|
||||||
// if (schedulerRemoveApplication == maxFailures) {
|
// Even though the completed container containerId3 event was sent to the
|
||||||
// synchronized (schedulerNotify) {
|
// earlier failed attempt, new RMAppAttempt can also capture this container
|
||||||
// schedulerNotify.addAndGet(1);
|
// info.
|
||||||
// schedulerNotify.notify();
|
// completed containerId4 is also transferred to the new attempt.
|
||||||
// }
|
RMAppAttempt newAttempt =
|
||||||
// }
|
app1.getRMAppAttempt(am2.getApplicationAttemptId());
|
||||||
// break;
|
// 4 containers finished, acquired/allocated/reserved/completed.
|
||||||
// default:
|
Assert.assertEquals(4, newAttempt.getJustFinishedContainers().size());
|
||||||
// break;
|
boolean container3Exists = false, container4Exists = false, container5Exists =
|
||||||
// }
|
false, container6Exists = false;
|
||||||
// }
|
for(ContainerStatus status : newAttempt.getJustFinishedContainers()) {
|
||||||
//
|
if(status.getContainerId().equals(containerId3)) {
|
||||||
// @Override
|
// containerId3 is the container ran by previous attempt but finished by the
|
||||||
// public QueueInfo getQueueInfo(String queueName,
|
// new attempt.
|
||||||
// boolean includeChildQueues,
|
container3Exists = true;
|
||||||
// boolean recursive) throws IOException {
|
}
|
||||||
// return null;
|
if (status.getContainerId().equals(containerId4)) {
|
||||||
// }
|
// containerId4 is the Acquired Container killed by the previous attempt,
|
||||||
// @Override
|
// it's now inside new attempt's finished container list.
|
||||||
// public List<QueueUserACLInfo> getQueueUserAclInfo() {
|
container4Exists = true;
|
||||||
// return null;
|
}
|
||||||
// }
|
if (status.getContainerId().equals(containerId5)) {
|
||||||
// @Override
|
// containerId5 is the Allocated container killed by previous failed attempt.
|
||||||
// public void addApplication(ApplicationId applicationId,
|
container5Exists = true;
|
||||||
// ApplicationMaster master, String user, String queue, Priority priority,
|
}
|
||||||
// ApplicationStore store)
|
if (status.getContainerId().equals(containerId6)) {
|
||||||
// throws IOException {
|
// containerId6 is the reserved container killed by previous failed attempt.
|
||||||
// }
|
container6Exists = true;
|
||||||
// @Override
|
}
|
||||||
// public void addNode(RMNode nodeInfo) {
|
}
|
||||||
// }
|
Assert.assertTrue(container3Exists && container4Exists && container5Exists
|
||||||
// @Override
|
&& container6Exists);
|
||||||
// public void recover(RMState state) throws Exception {
|
|
||||||
// }
|
// New SchedulerApplicationAttempt also has the containers info.
|
||||||
// @Override
|
rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
|
||||||
// public void reinitialize(Configuration conf,
|
|
||||||
// ContainerTokenSecretManager secretManager, RMContext rmContext)
|
// record the scheduler attempt for testing.
|
||||||
// throws IOException {
|
SchedulerApplicationAttempt schedulerNewAttempt =
|
||||||
// }
|
((CapacityScheduler) rm1.getResourceScheduler())
|
||||||
//
|
.getCurrentAttemptForContainer(containerId2);
|
||||||
// @Override
|
// finish this application
|
||||||
// public void nodeUpdate(RMNode nodeInfo,
|
MockRM.finishApplicationMaster(app1, rm1, nm1, am2);
|
||||||
// Map<String, List<Container>> containers) {
|
|
||||||
// }
|
// the 2nd attempt released the 1st attempt's running container, when the
|
||||||
//
|
// 2nd attempt finishes.
|
||||||
// @Override
|
Assert.assertFalse(schedulerNewAttempt.getLiveContainers().contains(
|
||||||
// public Resource getMaximumResourceCapability() {
|
containerId2));
|
||||||
// // TODO Auto-generated method stub
|
// all 4 normal containers finished.
|
||||||
// return null;
|
Assert.assertEquals(5, newAttempt.getJustFinishedContainers().size());
|
||||||
// }
|
|
||||||
//
|
rm1.stop();
|
||||||
// @Override
|
}
|
||||||
// public Resource getMinimumResourceCapability() {
|
|
||||||
// // TODO Auto-generated method stub
|
|
||||||
// return null;
|
|
||||||
// }
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// @Before
|
|
||||||
// public void setUp() {
|
|
||||||
//
|
|
||||||
// asmContext.getDispatcher().register(ApplicationEventType.class,
|
|
||||||
// new ResourceManager.ApplicationEventDispatcher(asmContext));
|
|
||||||
//
|
|
||||||
// appID = recordFactory.newRecordInstance(ApplicationId.class);
|
|
||||||
// appID.setClusterTimestamp(System.currentTimeMillis());
|
|
||||||
// appID.setId(1);
|
|
||||||
// Configuration conf = new Configuration();
|
|
||||||
// scheduler = new DummyResourceScheduler();
|
|
||||||
// asmContext.getDispatcher().init(conf);
|
|
||||||
// asmContext.getDispatcher().start();
|
|
||||||
// asmContext.getDispatcher().register(ApplicationTrackerEventType.class, scheduler);
|
|
||||||
// appImpl = new ExtApplicationsManagerImpl(appTokenSecretManager, scheduler, asmContext);
|
|
||||||
//
|
|
||||||
// conf.setLong(YarnConfiguration.AM_EXPIRY_INTERVAL, 1000L);
|
|
||||||
// conf.setInt(RMConfig.AM_MAX_RETRIES, maxFailures);
|
|
||||||
// appImpl.init(conf);
|
|
||||||
// appImpl.start();
|
|
||||||
//
|
|
||||||
// this.clientRMService = new ClientRMService(asmContext, appImpl
|
|
||||||
// .getAmLivelinessMonitor(), appImpl.getClientToAMSecretManager(),
|
|
||||||
// scheduler);
|
|
||||||
// this.clientRMService.init(conf);
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// @After
|
|
||||||
// public void tearDown() {
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// private void waitForFailed(AppAttempt application, ApplicationState
|
|
||||||
// finalState) throws Exception {
|
|
||||||
// int count = 0;
|
|
||||||
// while(application.getState() != finalState && count < 10) {
|
|
||||||
// Thread.sleep(500);
|
|
||||||
// count++;
|
|
||||||
// }
|
|
||||||
// Assert.assertEquals(finalState, application.getState());
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// @Test
|
|
||||||
// public void testAMRestart() throws Exception {
|
|
||||||
// ApplicationSubmissionContext subContext = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
|
|
||||||
// subContext.setApplicationId(appID);
|
|
||||||
// subContext.setApplicationName("dummyApp");
|
|
||||||
//// subContext.command = new ArrayList<String>();
|
|
||||||
//// subContext.environment = new HashMap<String, String>();
|
|
||||||
//// subContext.fsTokens = new ArrayList<String>();
|
|
||||||
// subContext.setFsTokensTodo(ByteBuffer.wrap(new byte[0]));
|
|
||||||
// SubmitApplicationRequest request = recordFactory
|
|
||||||
// .newRecordInstance(SubmitApplicationRequest.class);
|
|
||||||
// request.setApplicationSubmissionContext(subContext);
|
|
||||||
// clientRMService.submitApplication(request);
|
|
||||||
// AppAttempt application = asmContext.getApplications().get(appID);
|
|
||||||
// synchronized (schedulerNotify) {
|
|
||||||
// while(schedulerNotify.get() == 0) {
|
|
||||||
// schedulerNotify.wait();
|
|
||||||
// }
|
|
||||||
// }
|
|
||||||
// Assert.assertEquals(maxFailures, launcherCleanupCalled);
|
|
||||||
// Assert.assertEquals(maxFailures, launcherLaunchCalled);
|
|
||||||
// Assert.assertEquals(maxFailures, schedulerAddApplication);
|
|
||||||
// Assert.assertEquals(maxFailures, schedulerRemoveApplication);
|
|
||||||
// Assert.assertEquals(maxFailures, application.getFailedCount());
|
|
||||||
// waitForFailed(application, ApplicationState.FAILED);
|
|
||||||
// stop = true;
|
|
||||||
// }
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -460,7 +460,7 @@ public class TestRMAppTransitions {
|
||||||
LOG.info("--- START: testUnmanagedAppFailPath ---");
|
LOG.info("--- START: testUnmanagedAppFailPath ---");
|
||||||
application = testCreateAppRunning(subContext);
|
application = testCreateAppRunning(subContext);
|
||||||
RMAppEvent event = new RMAppFailedAttemptEvent(
|
RMAppEvent event = new RMAppFailedAttemptEvent(
|
||||||
application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED, "");
|
application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED, "", false);
|
||||||
application.handle(event);
|
application.handle(event);
|
||||||
rmDispatcher.await();
|
rmDispatcher.await();
|
||||||
RMAppAttempt appAttempt = application.getCurrentAppAttempt();
|
RMAppAttempt appAttempt = application.getCurrentAppAttempt();
|
||||||
|
@ -582,7 +582,7 @@ public class TestRMAppTransitions {
|
||||||
for (int i=1; i < maxAppAttempts; i++) {
|
for (int i=1; i < maxAppAttempts; i++) {
|
||||||
RMAppEvent event =
|
RMAppEvent event =
|
||||||
new RMAppFailedAttemptEvent(application.getApplicationId(),
|
new RMAppFailedAttemptEvent(application.getApplicationId(),
|
||||||
RMAppEventType.ATTEMPT_FAILED, "");
|
RMAppEventType.ATTEMPT_FAILED, "", false);
|
||||||
application.handle(event);
|
application.handle(event);
|
||||||
assertAppState(RMAppState.ACCEPTED, application);
|
assertAppState(RMAppState.ACCEPTED, application);
|
||||||
event =
|
event =
|
||||||
|
@ -598,7 +598,7 @@ public class TestRMAppTransitions {
|
||||||
String message = "Test fail";
|
String message = "Test fail";
|
||||||
RMAppEvent event =
|
RMAppEvent event =
|
||||||
new RMAppFailedAttemptEvent(application.getApplicationId(),
|
new RMAppFailedAttemptEvent(application.getApplicationId(),
|
||||||
RMAppEventType.ATTEMPT_FAILED, message);
|
RMAppEventType.ATTEMPT_FAILED, message, false);
|
||||||
application.handle(event);
|
application.handle(event);
|
||||||
rmDispatcher.await();
|
rmDispatcher.await();
|
||||||
sendAppUpdateSavedEvent(application);
|
sendAppUpdateSavedEvent(application);
|
||||||
|
@ -655,7 +655,7 @@ public class TestRMAppTransitions {
|
||||||
for (int i=1; i<maxAppAttempts; i++) {
|
for (int i=1; i<maxAppAttempts; i++) {
|
||||||
RMAppEvent event =
|
RMAppEvent event =
|
||||||
new RMAppFailedAttemptEvent(application.getApplicationId(),
|
new RMAppFailedAttemptEvent(application.getApplicationId(),
|
||||||
RMAppEventType.ATTEMPT_FAILED, "");
|
RMAppEventType.ATTEMPT_FAILED, "", false);
|
||||||
application.handle(event);
|
application.handle(event);
|
||||||
rmDispatcher.await();
|
rmDispatcher.await();
|
||||||
assertAppState(RMAppState.ACCEPTED, application);
|
assertAppState(RMAppState.ACCEPTED, application);
|
||||||
|
@ -680,7 +680,7 @@ public class TestRMAppTransitions {
|
||||||
// after max application attempts
|
// after max application attempts
|
||||||
RMAppEvent event =
|
RMAppEvent event =
|
||||||
new RMAppFailedAttemptEvent(application.getApplicationId(),
|
new RMAppFailedAttemptEvent(application.getApplicationId(),
|
||||||
RMAppEventType.ATTEMPT_FAILED, "");
|
RMAppEventType.ATTEMPT_FAILED, "", false);
|
||||||
application.handle(event);
|
application.handle(event);
|
||||||
rmDispatcher.await();
|
rmDispatcher.await();
|
||||||
sendAppUpdateSavedEvent(application);
|
sendAppUpdateSavedEvent(application);
|
||||||
|
@ -804,7 +804,7 @@ public class TestRMAppTransitions {
|
||||||
// KILLED => KILLED event RMAppEventType.ATTEMPT_FAILED
|
// KILLED => KILLED event RMAppEventType.ATTEMPT_FAILED
|
||||||
event =
|
event =
|
||||||
new RMAppFailedAttemptEvent(application.getApplicationId(),
|
new RMAppFailedAttemptEvent(application.getApplicationId(),
|
||||||
RMAppEventType.ATTEMPT_FAILED, "");
|
RMAppEventType.ATTEMPT_FAILED, "", false);
|
||||||
application.handle(event);
|
application.handle(event);
|
||||||
rmDispatcher.await();
|
rmDispatcher.await();
|
||||||
assertTimesAtFinish(application);
|
assertTimesAtFinish(application);
|
||||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||||
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.ContainerLaunchContext;
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerState;
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||||
|
@ -68,10 +69,10 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventT
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
|
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState;
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState;
|
||||||
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.RMAppEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
|
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.RMAppImpl;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
|
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.RMAppAttemptContainerAllocatedEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
|
||||||
|
@ -120,13 +121,14 @@ public class TestRMAppAttemptTransitions {
|
||||||
private AMLivelinessMonitor amFinishingMonitor;
|
private AMLivelinessMonitor amFinishingMonitor;
|
||||||
private RMStateStore store;
|
private RMStateStore store;
|
||||||
|
|
||||||
private RMApp application;
|
private RMAppImpl application;
|
||||||
private RMAppAttempt applicationAttempt;
|
private RMAppAttempt applicationAttempt;
|
||||||
|
|
||||||
private Configuration conf = new Configuration();
|
private Configuration conf = new Configuration();
|
||||||
private AMRMTokenSecretManager amRMTokenManager = spy(new AMRMTokenSecretManager(conf));
|
private AMRMTokenSecretManager amRMTokenManager = spy(new AMRMTokenSecretManager(conf));
|
||||||
private ClientToAMTokenSecretManagerInRM clientToAMTokenManager =
|
private ClientToAMTokenSecretManagerInRM clientToAMTokenManager =
|
||||||
spy(new ClientToAMTokenSecretManagerInRM());
|
spy(new ClientToAMTokenSecretManagerInRM());
|
||||||
|
private boolean transferStateFromPreviousAttempt = false;
|
||||||
|
|
||||||
private final class TestApplicationAttemptEventDispatcher implements
|
private final class TestApplicationAttemptEventDispatcher implements
|
||||||
EventHandler<RMAppAttemptEvent> {
|
EventHandler<RMAppAttemptEvent> {
|
||||||
|
@ -150,6 +152,11 @@ public class TestRMAppAttemptTransitions {
|
||||||
@Override
|
@Override
|
||||||
public void handle(RMAppEvent event) {
|
public void handle(RMAppEvent event) {
|
||||||
assertEquals(application.getApplicationId(), event.getApplicationId());
|
assertEquals(application.getApplicationId(), event.getApplicationId());
|
||||||
|
if (event instanceof RMAppFailedAttemptEvent) {
|
||||||
|
transferStateFromPreviousAttempt =
|
||||||
|
((RMAppFailedAttemptEvent) event)
|
||||||
|
.getTransferStateFromPreviousAttempt();
|
||||||
|
}
|
||||||
try {
|
try {
|
||||||
application.handle(event);
|
application.handle(event);
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
|
@ -254,10 +261,10 @@ public class TestRMAppAttemptTransitions {
|
||||||
|
|
||||||
unmanagedAM = false;
|
unmanagedAM = false;
|
||||||
|
|
||||||
application = mock(RMApp.class);
|
application = mock(RMAppImpl.class);
|
||||||
applicationAttempt =
|
applicationAttempt =
|
||||||
new RMAppAttemptImpl(applicationAttemptId, rmContext, scheduler,
|
new RMAppAttemptImpl(applicationAttemptId, rmContext, scheduler,
|
||||||
masterService, submissionContext, new Configuration());
|
masterService, submissionContext, new Configuration(), false);
|
||||||
when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt);
|
when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt);
|
||||||
when(application.getApplicationId()).thenReturn(applicationId);
|
when(application.getApplicationId()).thenReturn(applicationId);
|
||||||
|
|
||||||
|
@ -371,6 +378,7 @@ public class TestRMAppAttemptTransitions {
|
||||||
assertNull(applicationAttempt.getFinalApplicationStatus());
|
assertNull(applicationAttempt.getFinalApplicationStatus());
|
||||||
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
|
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
|
||||||
verifyAttemptFinalStateSaved();
|
verifyAttemptFinalStateSaved();
|
||||||
|
assertFalse(transferStateFromPreviousAttempt);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -525,6 +533,7 @@ public class TestRMAppAttemptTransitions {
|
||||||
assertEquals(container, applicationAttempt.getMasterContainer());
|
assertEquals(container, applicationAttempt.getMasterContainer());
|
||||||
assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus());
|
assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus());
|
||||||
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
|
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
|
||||||
|
assertFalse(transferStateFromPreviousAttempt);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -654,6 +663,7 @@ public class TestRMAppAttemptTransitions {
|
||||||
diagnostics));
|
diagnostics));
|
||||||
testAppAttemptFinishedState(null, finalStatus, url, diagnostics, 1,
|
testAppAttemptFinishedState(null, finalStatus, url, diagnostics, 1,
|
||||||
true);
|
true);
|
||||||
|
assertFalse(transferStateFromPreviousAttempt);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void sendAttemptUpdateSavedEvent(RMAppAttempt applicationAttempt) {
|
private void sendAttemptUpdateSavedEvent(RMAppAttempt applicationAttempt) {
|
||||||
|
@ -681,6 +691,21 @@ public class TestRMAppAttemptTransitions {
|
||||||
"Unmanaged AM must register after AM attempt reaches LAUNCHED state.");
|
"Unmanaged AM must register after AM attempt reaches LAUNCHED state.");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUnmanagedAMContainersCleanup() {
|
||||||
|
unmanagedAM = true;
|
||||||
|
when(submissionContext.getUnmanagedAM()).thenReturn(true);
|
||||||
|
when(submissionContext.getKeepContainersAcrossApplicationAttempts())
|
||||||
|
.thenReturn(true);
|
||||||
|
// submit AM and check it goes to SUBMITTED state
|
||||||
|
submitApplicationAttempt();
|
||||||
|
// launch AM and verify attempt failed
|
||||||
|
applicationAttempt.handle(new RMAppAttemptRegistrationEvent(
|
||||||
|
applicationAttempt.getAppAttemptId(), "host", 8042, "oldtrackingurl"));
|
||||||
|
sendAttemptUpdateSavedEvent(applicationAttempt);
|
||||||
|
assertFalse(transferStateFromPreviousAttempt);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testNewToKilled() {
|
public void testNewToKilled() {
|
||||||
applicationAttempt.handle(
|
applicationAttempt.handle(
|
||||||
|
@ -1092,6 +1117,64 @@ public class TestRMAppAttemptTransitions {
|
||||||
Assert.assertNull(token);
|
Assert.assertNull(token);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFailedToFailed() {
|
||||||
|
// create a failed attempt.
|
||||||
|
when(submissionContext.getKeepContainersAcrossApplicationAttempts())
|
||||||
|
.thenReturn(true);
|
||||||
|
Container amContainer = allocateApplicationAttempt();
|
||||||
|
launchApplicationAttempt(amContainer);
|
||||||
|
runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
|
||||||
|
ContainerStatus cs1 =
|
||||||
|
ContainerStatus.newInstance(amContainer.getId(),
|
||||||
|
ContainerState.COMPLETE, "some error", 123);
|
||||||
|
ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
|
||||||
|
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
|
||||||
|
appAttemptId, cs1));
|
||||||
|
sendAttemptUpdateSavedEvent(applicationAttempt);
|
||||||
|
assertEquals(RMAppAttemptState.FAILED,
|
||||||
|
applicationAttempt.getAppAttemptState());
|
||||||
|
// should not kill containers when attempt fails.
|
||||||
|
assertTrue(transferStateFromPreviousAttempt);
|
||||||
|
|
||||||
|
// failed attempt captured the container finished event.
|
||||||
|
assertEquals(0, applicationAttempt.getJustFinishedContainers().size());
|
||||||
|
ContainerStatus cs2 =
|
||||||
|
ContainerStatus.newInstance(ContainerId.newInstance(appAttemptId, 2),
|
||||||
|
ContainerState.COMPLETE, "", 0);
|
||||||
|
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
|
||||||
|
appAttemptId, cs2));
|
||||||
|
assertEquals(1, applicationAttempt.getJustFinishedContainers().size());
|
||||||
|
assertEquals(cs2.getContainerId(), applicationAttempt
|
||||||
|
.getJustFinishedContainers().get(0).getContainerId());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testContainersCleanupForLastAttempt() {
|
||||||
|
// create a failed attempt.
|
||||||
|
applicationAttempt =
|
||||||
|
new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(), rmContext,
|
||||||
|
scheduler, masterService, submissionContext, new Configuration(),
|
||||||
|
true);
|
||||||
|
when(submissionContext.getKeepContainersAcrossApplicationAttempts())
|
||||||
|
.thenReturn(true);
|
||||||
|
when(submissionContext.getMaxAppAttempts()).thenReturn(1);
|
||||||
|
Container amContainer = allocateApplicationAttempt();
|
||||||
|
launchApplicationAttempt(amContainer);
|
||||||
|
runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
|
||||||
|
ContainerStatus cs1 =
|
||||||
|
ContainerStatus.newInstance(amContainer.getId(),
|
||||||
|
ContainerState.COMPLETE, "some error", 123);
|
||||||
|
ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
|
||||||
|
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
|
||||||
|
appAttemptId, cs1));
|
||||||
|
sendAttemptUpdateSavedEvent(applicationAttempt);
|
||||||
|
assertEquals(RMAppAttemptState.FAILED,
|
||||||
|
applicationAttempt.getAppAttemptState());
|
||||||
|
assertFalse(transferStateFromPreviousAttempt);
|
||||||
|
}
|
||||||
|
|
||||||
private void verifyTokenCount(ApplicationAttemptId appAttemptId, int count) {
|
private void verifyTokenCount(ApplicationAttemptId appAttemptId, int count) {
|
||||||
verify(amRMTokenManager, times(count)).applicationMasterFinished(appAttemptId);
|
verify(amRMTokenManager, times(count)).applicationMasterFinished(appAttemptId);
|
||||||
if (UserGroupInformation.isSecurityEnabled()) {
|
if (UserGroupInformation.isSecurityEnabled()) {
|
||||||
|
|
|
@ -562,18 +562,18 @@ public class TestCapacityScheduler {
|
||||||
new AppAddedSchedulerEvent(appId, "default", "user");
|
new AppAddedSchedulerEvent(appId, "default", "user");
|
||||||
cs.handle(addAppEvent);
|
cs.handle(addAppEvent);
|
||||||
SchedulerEvent addAttemptEvent =
|
SchedulerEvent addAttemptEvent =
|
||||||
new AppAttemptAddedSchedulerEvent(appAttemptId);
|
new AppAttemptAddedSchedulerEvent(appAttemptId, false);
|
||||||
cs.handle(addAttemptEvent);
|
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(),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
Collections.singletonList(host), null);
|
Collections.singletonList(host), null);
|
||||||
Assert.assertTrue(cs.getApplication(appAttemptId).isBlacklisted(host));
|
Assert.assertTrue(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
|
||||||
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
||||||
Collections.<ContainerId>emptyList(), null,
|
Collections.<ContainerId>emptyList(), null,
|
||||||
Collections.singletonList(host));
|
Collections.singletonList(host));
|
||||||
Assert.assertFalse(cs.getApplication(appAttemptId).isBlacklisted(host));
|
Assert.assertFalse(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
|
||||||
rm.stop();
|
rm.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -598,66 +598,6 @@ public class TestCapacityScheduler {
|
||||||
assertTrue(appComparator.compare(app2, app3) < 0);
|
assertTrue(appComparator.compare(app2, app3) < 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testConcurrentAccessOnApplications() throws Exception {
|
|
||||||
CapacityScheduler cs = new CapacityScheduler();
|
|
||||||
verifyConcurrentAccessOnApplications(
|
|
||||||
cs.appAttempts, FiCaSchedulerApp.class, Queue.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static <T extends SchedulerApplicationAttempt, Q extends Queue>
|
|
||||||
void verifyConcurrentAccessOnApplications(
|
|
||||||
final Map<ApplicationAttemptId, T> applications, Class<T> appClazz,
|
|
||||||
final Class<Q> queueClazz)
|
|
||||||
throws Exception {
|
|
||||||
final int size = 10000;
|
|
||||||
final ApplicationId appId = ApplicationId.newInstance(0, 0);
|
|
||||||
final Constructor<T> ctor = appClazz.getDeclaredConstructor(
|
|
||||||
ApplicationAttemptId.class, String.class, queueClazz,
|
|
||||||
ActiveUsersManager.class, RMContext.class);
|
|
||||||
|
|
||||||
ApplicationAttemptId appAttemptId0
|
|
||||||
= ApplicationAttemptId.newInstance(appId, 0);
|
|
||||||
applications.put(appAttemptId0, ctor.newInstance(
|
|
||||||
appAttemptId0, null, mock(queueClazz), null, null));
|
|
||||||
assertNotNull(applications.get(appAttemptId0));
|
|
||||||
|
|
||||||
// Imitating the thread of scheduler that will add and remove apps
|
|
||||||
final AtomicBoolean finished = new AtomicBoolean(false);
|
|
||||||
final AtomicBoolean failed = new AtomicBoolean(false);
|
|
||||||
Thread t = new Thread() {
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
for (int i = 1; i <= size; ++i) {
|
|
||||||
ApplicationAttemptId appAttemptId
|
|
||||||
= ApplicationAttemptId.newInstance(appId, i);
|
|
||||||
try {
|
|
||||||
applications.put(appAttemptId, ctor.newInstance(
|
|
||||||
appAttemptId, null, mock(queueClazz), null, null));
|
|
||||||
} catch (Exception e) {
|
|
||||||
failed.set(true);
|
|
||||||
finished.set(true);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
for (int i = 1; i <= size; ++i) {
|
|
||||||
ApplicationAttemptId appAttemptId
|
|
||||||
= ApplicationAttemptId.newInstance(appId, i);
|
|
||||||
applications.remove(appAttemptId);
|
|
||||||
}
|
|
||||||
finished.set(true);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
t.start();
|
|
||||||
|
|
||||||
// Imitating the thread of rmappattempt that will get the app
|
|
||||||
while (!finished.get()) {
|
|
||||||
assertNotNull(applications.get(appAttemptId0));
|
|
||||||
}
|
|
||||||
assertFalse(failed.get());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGetAppsInQueue() throws Exception {
|
public void testGetAppsInQueue() throws Exception {
|
||||||
Application application_0 = new Application("user_0", "a1", resourceManager);
|
Application application_0 = new Application("user_0", "a1", resourceManager);
|
||||||
|
|
|
@ -63,6 +63,8 @@ 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.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.AppAttemptRemovedSchedulerEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
|
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
|
||||||
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
||||||
|
@ -345,11 +347,16 @@ 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.submitApplicationAttempt(app_0, user_0);
|
AppAddedSchedulerEvent addAppEvent =
|
||||||
|
new AppAddedSchedulerEvent(appAttemptId_0.getApplicationId(),
|
||||||
|
a.getQueueName(), user_0);
|
||||||
|
cs.handle(addAppEvent);
|
||||||
|
AppAttemptAddedSchedulerEvent addAttemptEvent =
|
||||||
|
new AppAttemptAddedSchedulerEvent(appAttemptId_0, false);
|
||||||
|
cs.handle(addAttemptEvent);
|
||||||
|
|
||||||
when(cs.getApplication(appAttemptId_0)).thenReturn(app_0);
|
|
||||||
AppAttemptRemovedSchedulerEvent event = new AppAttemptRemovedSchedulerEvent(
|
AppAttemptRemovedSchedulerEvent event = new AppAttemptRemovedSchedulerEvent(
|
||||||
appAttemptId_0, RMAppAttemptState.FAILED);
|
appAttemptId_0, RMAppAttemptState.FAILED, false);
|
||||||
cs.handle(event);
|
cs.handle(event);
|
||||||
|
|
||||||
assertEquals(0, a.getMetrics().getAppsPending());
|
assertEquals(0, a.getMetrics().getAppsPending());
|
||||||
|
@ -365,9 +372,8 @@ public class TestLeafQueue {
|
||||||
assertEquals(1, a.getMetrics().getAppsSubmitted());
|
assertEquals(1, a.getMetrics().getAppsSubmitted());
|
||||||
assertEquals(1, a.getMetrics().getAppsPending());
|
assertEquals(1, a.getMetrics().getAppsPending());
|
||||||
|
|
||||||
when(cs.getApplication(appAttemptId_1)).thenReturn(app_0);
|
|
||||||
event = new AppAttemptRemovedSchedulerEvent(appAttemptId_0,
|
event = new AppAttemptRemovedSchedulerEvent(appAttemptId_0,
|
||||||
RMAppAttemptState.FINISHED);
|
RMAppAttemptState.FINISHED, false);
|
||||||
cs.handle(event);
|
cs.handle(event);
|
||||||
|
|
||||||
assertEquals(1, a.getMetrics().getAppsSubmitted());
|
assertEquals(1, a.getMetrics().getAppsSubmitted());
|
||||||
|
|
|
@ -261,7 +261,7 @@ public class TestFairScheduler {
|
||||||
// This conditional is for testAclSubmitApplication where app is rejected
|
// This conditional is for testAclSubmitApplication where app is rejected
|
||||||
// and no app is added.
|
// and no app is added.
|
||||||
if (scheduler.applications.containsKey(id.getApplicationId())) {
|
if (scheduler.applications.containsKey(id.getApplicationId())) {
|
||||||
scheduler.addApplicationAttempt(id);
|
scheduler.addApplicationAttempt(id, false);
|
||||||
}
|
}
|
||||||
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,
|
||||||
|
@ -590,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.appAttempts.get(attId).getCurrentReservation().getMemory());
|
assertEquals(1024, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory());
|
||||||
|
|
||||||
// Now another node checks in with capacity
|
// Now another node checks in with capacity
|
||||||
RMNode node2 =
|
RMNode node2 =
|
||||||
|
@ -606,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.appAttempts.get(attId).getCurrentReservation().getMemory());
|
assertEquals(1024, scheduler.getSchedulerApp(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.appAttempts.get(attId).getCurrentReservation().getMemory());
|
assertEquals(0, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory());
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -630,7 +630,7 @@ public class TestFairScheduler {
|
||||||
"user1");
|
"user1");
|
||||||
scheduler.handle(appAddedEvent);
|
scheduler.handle(appAddedEvent);
|
||||||
AppAttemptAddedSchedulerEvent attempAddedEvent =
|
AppAttemptAddedSchedulerEvent attempAddedEvent =
|
||||||
new AppAttemptAddedSchedulerEvent(appAttemptId);
|
new AppAttemptAddedSchedulerEvent(appAttemptId, false);
|
||||||
scheduler.handle(attempAddedEvent);
|
scheduler.handle(attempAddedEvent);
|
||||||
assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true)
|
assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true)
|
||||||
.getRunnableAppSchedulables().size());
|
.getRunnableAppSchedulables().size());
|
||||||
|
@ -656,7 +656,7 @@ public class TestFairScheduler {
|
||||||
"user2");
|
"user2");
|
||||||
scheduler.handle(appAddedEvent);
|
scheduler.handle(appAddedEvent);
|
||||||
AppAttemptAddedSchedulerEvent attempAddedEvent =
|
AppAttemptAddedSchedulerEvent attempAddedEvent =
|
||||||
new AppAttemptAddedSchedulerEvent(appAttemptId);
|
new AppAttemptAddedSchedulerEvent(appAttemptId, false);
|
||||||
scheduler.handle(attempAddedEvent);
|
scheduler.handle(attempAddedEvent);
|
||||||
assertEquals(0, scheduler.getQueueManager().getLeafQueue("user1", true)
|
assertEquals(0, scheduler.getQueueManager().getLeafQueue("user1", true)
|
||||||
.getRunnableAppSchedulables().size());
|
.getRunnableAppSchedulables().size());
|
||||||
|
@ -710,7 +710,6 @@ public class TestFairScheduler {
|
||||||
scheduler.reinitialize(conf, resourceManager.getRMContext());
|
scheduler.reinitialize(conf, resourceManager.getRMContext());
|
||||||
|
|
||||||
ApplicationAttemptId appId;
|
ApplicationAttemptId appId;
|
||||||
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));
|
||||||
|
@ -723,17 +722,17 @@ public class TestFairScheduler {
|
||||||
scheduler.getAllocationConfiguration().placementPolicy =
|
scheduler.getAllocationConfiguration().placementPolicy =
|
||||||
new QueuePlacementPolicy(rules, queues, conf);
|
new QueuePlacementPolicy(rules, queues, conf);
|
||||||
appId = createSchedulingRequest(1024, "somequeue", "user1");
|
appId = createSchedulingRequest(1024, "somequeue", "user1");
|
||||||
assertEquals("root.somequeue", apps.get(appId).getQueueName());
|
assertEquals("root.somequeue", scheduler.getSchedulerApp(appId).getQueueName());
|
||||||
appId = createSchedulingRequest(1024, "default", "user1");
|
appId = createSchedulingRequest(1024, "default", "user1");
|
||||||
assertEquals("root.user1", apps.get(appId).getQueueName());
|
assertEquals("root.user1", scheduler.getSchedulerApp(appId).getQueueName());
|
||||||
appId = createSchedulingRequest(1024, "default", "user3");
|
appId = createSchedulingRequest(1024, "default", "user3");
|
||||||
assertEquals("root.user3group", apps.get(appId).getQueueName());
|
assertEquals("root.user3group", scheduler.getSchedulerApp(appId).getQueueName());
|
||||||
appId = createSchedulingRequest(1024, "default", "user4");
|
appId = createSchedulingRequest(1024, "default", "user4");
|
||||||
assertEquals("root.user4subgroup1", apps.get(appId).getQueueName());
|
assertEquals("root.user4subgroup1", scheduler.getSchedulerApp(appId).getQueueName());
|
||||||
appId = createSchedulingRequest(1024, "default", "user5");
|
appId = createSchedulingRequest(1024, "default", "user5");
|
||||||
assertEquals("root.user5subgroup2", apps.get(appId).getQueueName());
|
assertEquals("root.user5subgroup2", scheduler.getSchedulerApp(appId).getQueueName());
|
||||||
appId = createSchedulingRequest(1024, "default", "otheruser");
|
appId = createSchedulingRequest(1024, "default", "otheruser");
|
||||||
assertEquals("root.default", apps.get(appId).getQueueName());
|
assertEquals("root.default", scheduler.getSchedulerApp(appId).getQueueName());
|
||||||
|
|
||||||
// test without specified as first rule
|
// test without specified as first rule
|
||||||
rules = new ArrayList<QueuePlacementRule>();
|
rules = new ArrayList<QueuePlacementRule>();
|
||||||
|
@ -743,11 +742,11 @@ public class TestFairScheduler {
|
||||||
scheduler.getAllocationConfiguration().placementPolicy =
|
scheduler.getAllocationConfiguration().placementPolicy =
|
||||||
new QueuePlacementPolicy(rules, queues, conf);
|
new QueuePlacementPolicy(rules, queues, conf);
|
||||||
appId = createSchedulingRequest(1024, "somequeue", "user1");
|
appId = createSchedulingRequest(1024, "somequeue", "user1");
|
||||||
assertEquals("root.user1", apps.get(appId).getQueueName());
|
assertEquals("root.user1", scheduler.getSchedulerApp(appId).getQueueName());
|
||||||
appId = createSchedulingRequest(1024, "somequeue", "otheruser");
|
appId = createSchedulingRequest(1024, "somequeue", "otheruser");
|
||||||
assertEquals("root.somequeue", apps.get(appId).getQueueName());
|
assertEquals("root.somequeue", scheduler.getSchedulerApp(appId).getQueueName());
|
||||||
appId = createSchedulingRequest(1024, "default", "otheruser");
|
appId = createSchedulingRequest(1024, "default", "otheruser");
|
||||||
assertEquals("root.default", apps.get(appId).getQueueName());
|
assertEquals("root.default", scheduler.getSchedulerApp(appId).getQueueName());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -802,13 +801,13 @@ public class TestFairScheduler {
|
||||||
|
|
||||||
ApplicationAttemptId id11 = createAppAttemptId(1, 1);
|
ApplicationAttemptId id11 = createAppAttemptId(1, 1);
|
||||||
scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1");
|
scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1");
|
||||||
scheduler.addApplicationAttempt(id11);
|
scheduler.addApplicationAttempt(id11, false);
|
||||||
ApplicationAttemptId id21 = createAppAttemptId(2, 1);
|
ApplicationAttemptId id21 = createAppAttemptId(2, 1);
|
||||||
scheduler.addApplication(id21.getApplicationId(), "root.queue2", "user1");
|
scheduler.addApplication(id21.getApplicationId(), "root.queue2", "user1");
|
||||||
scheduler.addApplicationAttempt(id21);
|
scheduler.addApplicationAttempt(id21, false);
|
||||||
ApplicationAttemptId id22 = createAppAttemptId(2, 2);
|
ApplicationAttemptId id22 = createAppAttemptId(2, 2);
|
||||||
scheduler.addApplication(id22.getApplicationId(), "root.queue2", "user1");
|
scheduler.addApplication(id22.getApplicationId(), "root.queue2", "user1");
|
||||||
scheduler.addApplicationAttempt(id22);
|
scheduler.addApplicationAttempt(id22, false);
|
||||||
|
|
||||||
int minReqSize =
|
int minReqSize =
|
||||||
FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB;
|
FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB;
|
||||||
|
@ -854,7 +853,7 @@ public class TestFairScheduler {
|
||||||
"user1");
|
"user1");
|
||||||
scheduler.handle(appAddedEvent);
|
scheduler.handle(appAddedEvent);
|
||||||
AppAttemptAddedSchedulerEvent attemptAddedEvent =
|
AppAttemptAddedSchedulerEvent attemptAddedEvent =
|
||||||
new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1));
|
new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false);
|
||||||
scheduler.handle(attemptAddedEvent);
|
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)
|
||||||
|
@ -865,7 +864,7 @@ public class TestFairScheduler {
|
||||||
.getRunnableAppSchedulables().size());
|
.getRunnableAppSchedulables().size());
|
||||||
|
|
||||||
AppAttemptRemovedSchedulerEvent appRemovedEvent1 = new AppAttemptRemovedSchedulerEvent(
|
AppAttemptRemovedSchedulerEvent appRemovedEvent1 = new AppAttemptRemovedSchedulerEvent(
|
||||||
createAppAttemptId(1, 1), RMAppAttemptState.FINISHED);
|
createAppAttemptId(1, 1), RMAppAttemptState.FINISHED, false);
|
||||||
|
|
||||||
// Now remove app
|
// Now remove app
|
||||||
scheduler.handle(appRemovedEvent1);
|
scheduler.handle(appRemovedEvent1);
|
||||||
|
@ -1138,12 +1137,12 @@ public class TestFairScheduler {
|
||||||
scheduler.handle(nodeUpdate3);
|
scheduler.handle(nodeUpdate3);
|
||||||
}
|
}
|
||||||
|
|
||||||
assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
|
||||||
assertEquals(1, scheduler.appAttempts.get(app2).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(app2).getLiveContainers().size());
|
||||||
assertEquals(1, scheduler.appAttempts.get(app3).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(app3).getLiveContainers().size());
|
||||||
assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
|
||||||
assertEquals(1, scheduler.appAttempts.get(app5).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(app5).getLiveContainers().size());
|
||||||
assertEquals(1, scheduler.appAttempts.get(app6).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(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 =
|
||||||
|
@ -1166,16 +1165,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.appAttempts.get(app1).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
|
||||||
assertEquals(1, scheduler.appAttempts.get(app2).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(app2).getLiveContainers().size());
|
||||||
assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
|
||||||
assertEquals(1, scheduler.appAttempts.get(app5).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(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.appAttempts.get(app3).getLiveContainers(),
|
assertTrue(!Collections.disjoint(scheduler.getSchedulerApp(app3).getLiveContainers(),
|
||||||
scheduler.appAttempts.get(app3).getPreemptionContainers()));
|
scheduler.getSchedulerApp(app3).getPreemptionContainers()));
|
||||||
assertTrue(!Collections.disjoint(scheduler.appAttempts.get(app6).getLiveContainers(),
|
assertTrue(!Collections.disjoint(scheduler.getSchedulerApp(app6).getLiveContainers(),
|
||||||
scheduler.appAttempts.get(app6).getPreemptionContainers()));
|
scheduler.getSchedulerApp(app6).getPreemptionContainers()));
|
||||||
|
|
||||||
// Pretend 15 seconds have passed
|
// Pretend 15 seconds have passed
|
||||||
clock.tick(15);
|
clock.tick(15);
|
||||||
|
@ -1185,8 +1184,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.appAttempts.get(app6).getLiveContainers().size());
|
assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size());
|
||||||
assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size());
|
assertEquals(0, scheduler.getSchedulerApp(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(),
|
||||||
|
@ -1200,22 +1199,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.appAttempts.get(app1).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
|
||||||
assertEquals(0, scheduler.appAttempts.get(app2).getLiveContainers().size());
|
assertEquals(0, scheduler.getSchedulerApp(app2).getLiveContainers().size());
|
||||||
assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size());
|
assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size());
|
||||||
assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
|
||||||
assertEquals(0, scheduler.appAttempts.get(app5).getLiveContainers().size());
|
assertEquals(0, scheduler.getSchedulerApp(app5).getLiveContainers().size());
|
||||||
assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size());
|
assertEquals(0, scheduler.getSchedulerApp(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.appAttempts.get(app1).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size());
|
||||||
assertEquals(0, scheduler.appAttempts.get(app2).getLiveContainers().size());
|
assertEquals(0, scheduler.getSchedulerApp(app2).getLiveContainers().size());
|
||||||
assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size());
|
assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size());
|
||||||
assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size());
|
||||||
assertEquals(0, scheduler.appAttempts.get(app5).getLiveContainers().size());
|
assertEquals(0, scheduler.getSchedulerApp(app5).getLiveContainers().size());
|
||||||
assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size());
|
assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test (timeout = 5000)
|
@Test (timeout = 5000)
|
||||||
|
@ -1374,9 +1373,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.appAttempts.get(attId1).getCurrentReservation().getMemory());
|
scheduler.getSchedulerApp(attId1).getCurrentReservation().getMemory());
|
||||||
assertEquals(0,
|
assertEquals(0,
|
||||||
scheduler.appAttempts.get(attId2).getCurrentReservation().getMemory());
|
scheduler.getSchedulerApp(attId2).getCurrentReservation().getMemory());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test (timeout = 5000)
|
@Test (timeout = 5000)
|
||||||
|
@ -1411,7 +1410,7 @@ public class TestFairScheduler {
|
||||||
scheduler.handle(updateEvent);
|
scheduler.handle(updateEvent);
|
||||||
|
|
||||||
// App 1 should be running
|
// App 1 should be running
|
||||||
assertEquals(1, scheduler.appAttempts.get(attId1).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(attId1).getLiveContainers().size());
|
||||||
|
|
||||||
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
|
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
|
||||||
"user1", 1);
|
"user1", 1);
|
||||||
|
@ -1420,7 +1419,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.appAttempts.get(attId2).getLiveContainers().size());
|
assertEquals(0, scheduler.getSchedulerApp(attId2).getLiveContainers().size());
|
||||||
|
|
||||||
// Request another container for app 1
|
// Request another container for app 1
|
||||||
createSchedulingRequestExistingApplication(1024, 1, attId1);
|
createSchedulingRequestExistingApplication(1024, 1, attId1);
|
||||||
|
@ -1429,7 +1428,7 @@ public class TestFairScheduler {
|
||||||
scheduler.handle(updateEvent);
|
scheduler.handle(updateEvent);
|
||||||
|
|
||||||
// Request should be fulfilled
|
// Request should be fulfilled
|
||||||
assertEquals(2, scheduler.appAttempts.get(attId1).getLiveContainers().size());
|
assertEquals(2, scheduler.getSchedulerApp(attId1).getLiveContainers().size());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test (timeout = 5000)
|
@Test (timeout = 5000)
|
||||||
|
@ -1449,10 +1448,10 @@ public class TestFairScheduler {
|
||||||
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
||||||
scheduler.handle(updateEvent);
|
scheduler.handle(updateEvent);
|
||||||
|
|
||||||
FSSchedulerApp app = scheduler.appAttempts.get(attId);
|
FSSchedulerApp app = scheduler.getSchedulerApp(attId);
|
||||||
assertEquals(1, app.getLiveContainers().size());
|
assertEquals(1, app.getLiveContainers().size());
|
||||||
|
|
||||||
ContainerId containerId = scheduler.appAttempts.get(attId)
|
ContainerId containerId = scheduler.getSchedulerApp(attId)
|
||||||
.getLiveContainers().iterator().next().getContainerId();
|
.getLiveContainers().iterator().next().getContainerId();
|
||||||
|
|
||||||
// Cause reservation to be created
|
// Cause reservation to be created
|
||||||
|
@ -1521,9 +1520,9 @@ public class TestFairScheduler {
|
||||||
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
|
ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1",
|
||||||
"norealuserhasthisname2", 1);
|
"norealuserhasthisname2", 1);
|
||||||
|
|
||||||
FSSchedulerApp app1 = scheduler.appAttempts.get(attId1);
|
FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1);
|
||||||
assertNotNull("The application was not allowed", app1);
|
assertNotNull("The application was not allowed", app1);
|
||||||
FSSchedulerApp app2 = scheduler.appAttempts.get(attId2);
|
FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2);
|
||||||
assertNull("The application was allowed", app2);
|
assertNull("The application was allowed", app2);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1547,7 +1546,7 @@ public class TestFairScheduler {
|
||||||
|
|
||||||
ApplicationAttemptId appId = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
|
ApplicationAttemptId appId = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
|
||||||
scheduler.addApplication(appId.getApplicationId(), "queue1", "user1");
|
scheduler.addApplication(appId.getApplicationId(), "queue1", "user1");
|
||||||
scheduler.addApplicationAttempt(appId);
|
scheduler.addApplicationAttempt(appId, false);
|
||||||
|
|
||||||
// 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
|
||||||
|
@ -1566,14 +1565,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.appAttempts.get(appId).getLiveContainers().size());
|
assertEquals(1, scheduler.getSchedulerApp(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.appAttempts.get(appId).getLiveContainers().size());
|
assertEquals(2, scheduler.getSchedulerApp(appId).getLiveContainers().size());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test (timeout = 5000)
|
@Test (timeout = 5000)
|
||||||
|
@ -1592,8 +1591,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.appAttempts.get(attId1);
|
FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1);
|
||||||
FSSchedulerApp app2 = scheduler.appAttempts.get(attId2);
|
FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2);
|
||||||
|
|
||||||
FSLeafQueue queue1 = scheduler.getQueueManager().getLeafQueue("queue1", true);
|
FSLeafQueue queue1 = scheduler.getQueueManager().getLeafQueue("queue1", true);
|
||||||
queue1.setPolicy(new FifoPolicy());
|
queue1.setPolicy(new FifoPolicy());
|
||||||
|
@ -1633,7 +1632,7 @@ public class TestFairScheduler {
|
||||||
|
|
||||||
ApplicationAttemptId attId =
|
ApplicationAttemptId attId =
|
||||||
createSchedulingRequest(1024, "root.default", "user", 8);
|
createSchedulingRequest(1024, "root.default", "user", 8);
|
||||||
FSSchedulerApp app = scheduler.appAttempts.get(attId);
|
FSSchedulerApp app = scheduler.getSchedulerApp(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;
|
||||||
|
@ -1695,10 +1694,10 @@ public class TestFairScheduler {
|
||||||
ApplicationAttemptId attId4 =
|
ApplicationAttemptId attId4 =
|
||||||
createSchedulingRequest(1024, fifoQueue, user, 4);
|
createSchedulingRequest(1024, fifoQueue, user, 4);
|
||||||
|
|
||||||
FSSchedulerApp app1 = scheduler.appAttempts.get(attId1);
|
FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1);
|
||||||
FSSchedulerApp app2 = scheduler.appAttempts.get(attId2);
|
FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2);
|
||||||
FSSchedulerApp app3 = scheduler.appAttempts.get(attId3);
|
FSSchedulerApp app3 = scheduler.getSchedulerApp(attId3);
|
||||||
FSSchedulerApp app4 = scheduler.appAttempts.get(attId4);
|
FSSchedulerApp app4 = scheduler.getSchedulerApp(attId4);
|
||||||
|
|
||||||
scheduler.getQueueManager().getLeafQueue(fifoQueue, true)
|
scheduler.getQueueManager().getLeafQueue(fifoQueue, true)
|
||||||
.setPolicy(SchedulingPolicy.parse("fifo"));
|
.setPolicy(SchedulingPolicy.parse("fifo"));
|
||||||
|
@ -1813,7 +1812,7 @@ public class TestFairScheduler {
|
||||||
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
||||||
scheduler.handle(updateEvent);
|
scheduler.handle(updateEvent);
|
||||||
|
|
||||||
FSSchedulerApp app = scheduler.appAttempts.get(attId);
|
FSSchedulerApp app = scheduler.getSchedulerApp(attId);
|
||||||
assertEquals(0, app.getLiveContainers().size());
|
assertEquals(0, app.getLiveContainers().size());
|
||||||
assertEquals(0, app.getReservedContainers().size());
|
assertEquals(0, app.getReservedContainers().size());
|
||||||
|
|
||||||
|
@ -1882,7 +1881,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.appAttempts.get(attId1);
|
FSSchedulerApp app = scheduler.getSchedulerApp(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());
|
||||||
|
@ -1921,7 +1920,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.appAttempts.get(attId1);
|
FSSchedulerApp app = scheduler.getSchedulerApp(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());
|
||||||
|
@ -1954,7 +1953,7 @@ public class TestFairScheduler {
|
||||||
|
|
||||||
ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1",
|
ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1",
|
||||||
"user1", 0);
|
"user1", 0);
|
||||||
FSSchedulerApp app = scheduler.appAttempts.get(attId);
|
FSSchedulerApp app = scheduler.getSchedulerApp(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);
|
||||||
|
@ -1994,7 +1993,7 @@ public class TestFairScheduler {
|
||||||
|
|
||||||
ApplicationAttemptId attId = createSchedulingRequest(1024, 1, "default",
|
ApplicationAttemptId attId = createSchedulingRequest(1024, 1, "default",
|
||||||
"user1", 2);
|
"user1", 2);
|
||||||
FSSchedulerApp app = scheduler.appAttempts.get(attId);
|
FSSchedulerApp app = scheduler.getSchedulerApp(attId);
|
||||||
scheduler.update();
|
scheduler.update();
|
||||||
|
|
||||||
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
|
||||||
|
@ -2014,10 +2013,10 @@ public class TestFairScheduler {
|
||||||
|
|
||||||
ApplicationAttemptId appAttId1 = createSchedulingRequest(2048, 1, "queue1",
|
ApplicationAttemptId appAttId1 = createSchedulingRequest(2048, 1, "queue1",
|
||||||
"user1", 2);
|
"user1", 2);
|
||||||
FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1);
|
FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1);
|
||||||
ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 2, "queue1",
|
ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 2, "queue1",
|
||||||
"user1", 2);
|
"user1", 2);
|
||||||
FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2);
|
FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2);
|
||||||
|
|
||||||
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
|
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
|
||||||
drfPolicy.initialize(scheduler.getClusterCapacity());
|
drfPolicy.initialize(scheduler.getClusterCapacity());
|
||||||
|
@ -2055,13 +2054,13 @@ public class TestFairScheduler {
|
||||||
|
|
||||||
ApplicationAttemptId appAttId1 = createSchedulingRequest(3072, 1, "queue1",
|
ApplicationAttemptId appAttId1 = createSchedulingRequest(3072, 1, "queue1",
|
||||||
"user1", 2);
|
"user1", 2);
|
||||||
FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1);
|
FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1);
|
||||||
ApplicationAttemptId appAttId2 = createSchedulingRequest(2048, 2, "queue1",
|
ApplicationAttemptId appAttId2 = createSchedulingRequest(2048, 2, "queue1",
|
||||||
"user1", 2);
|
"user1", 2);
|
||||||
FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2);
|
FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2);
|
||||||
ApplicationAttemptId appAttId3 = createSchedulingRequest(1024, 2, "queue2",
|
ApplicationAttemptId appAttId3 = createSchedulingRequest(1024, 2, "queue2",
|
||||||
"user1", 2);
|
"user1", 2);
|
||||||
FSSchedulerApp app3 = scheduler.appAttempts.get(appAttId3);
|
FSSchedulerApp app3 = scheduler.getSchedulerApp(appAttId3);
|
||||||
|
|
||||||
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
|
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
|
||||||
drfPolicy.initialize(scheduler.getClusterCapacity());
|
drfPolicy.initialize(scheduler.getClusterCapacity());
|
||||||
|
@ -2092,19 +2091,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.appAttempts.get(appAttId1);
|
FSSchedulerApp app1 = scheduler.getSchedulerApp(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.appAttempts.get(appAttId2);
|
FSSchedulerApp app2 = scheduler.getSchedulerApp(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.appAttempts.get(appAttId3);
|
FSSchedulerApp app3 = scheduler.getSchedulerApp(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.appAttempts.get(appAttId4);
|
FSSchedulerApp app4 = scheduler.getSchedulerApp(appAttId4);
|
||||||
|
|
||||||
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
|
DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy();
|
||||||
drfPolicy.initialize(scheduler.getClusterCapacity());
|
drfPolicy.initialize(scheduler.getClusterCapacity());
|
||||||
|
@ -2184,7 +2183,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.appAttempts.get(attId1);
|
FSSchedulerApp app = scheduler.getSchedulerApp(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());
|
||||||
|
@ -2195,16 +2194,8 @@ public class TestFairScheduler {
|
||||||
assertEquals(1, app.getLiveContainers().size());
|
assertEquals(1, app.getLiveContainers().size());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testConcurrentAccessOnApplications() throws Exception {
|
|
||||||
FairScheduler fs = new FairScheduler();
|
|
||||||
TestCapacityScheduler.verifyConcurrentAccessOnApplications(
|
|
||||||
fs.appAttempts, FSSchedulerApp.class, FSLeafQueue.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
private void verifyAppRunnable(ApplicationAttemptId attId, boolean runnable) {
|
private void verifyAppRunnable(ApplicationAttemptId attId, boolean runnable) {
|
||||||
FSSchedulerApp app = scheduler.appAttempts.get(attId);
|
FSSchedulerApp app = scheduler.getSchedulerApp(attId);
|
||||||
FSLeafQueue queue = app.getQueue();
|
FSLeafQueue queue = app.getQueue();
|
||||||
Collection<AppSchedulable> runnableApps =
|
Collection<AppSchedulable> runnableApps =
|
||||||
queue.getRunnableAppSchedulables();
|
queue.getRunnableAppSchedulables();
|
||||||
|
@ -2260,7 +2251,7 @@ public class TestFairScheduler {
|
||||||
|
|
||||||
// Remove app 1 and both app 2 and app 4 should becomes runnable in its place
|
// Remove app 1 and both app 2 and app 4 should becomes runnable in its place
|
||||||
AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
|
AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
|
||||||
new AppAttemptRemovedSchedulerEvent(attId1, RMAppAttemptState.FINISHED);
|
new AppAttemptRemovedSchedulerEvent(attId1, RMAppAttemptState.FINISHED, false);
|
||||||
scheduler.handle(appRemovedEvent1);
|
scheduler.handle(appRemovedEvent1);
|
||||||
verifyAppRunnable(attId2, true);
|
verifyAppRunnable(attId2, true);
|
||||||
verifyQueueNumRunnable("queue2", 1, 0);
|
verifyQueueNumRunnable("queue2", 1, 0);
|
||||||
|
@ -2324,7 +2315,7 @@ public class TestFairScheduler {
|
||||||
// Even though the app was removed from sub3, the app from sub2 gets to go
|
// Even though the app was removed from sub3, the app from sub2 gets to go
|
||||||
// because it came in first
|
// because it came in first
|
||||||
AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
|
AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
|
||||||
new AppAttemptRemovedSchedulerEvent(attId2, RMAppAttemptState.FINISHED);
|
new AppAttemptRemovedSchedulerEvent(attId2, RMAppAttemptState.FINISHED, false);
|
||||||
scheduler.handle(appRemovedEvent1);
|
scheduler.handle(appRemovedEvent1);
|
||||||
verifyAppRunnable(attId4, true);
|
verifyAppRunnable(attId4, true);
|
||||||
verifyQueueNumRunnable("queue1.sub2", 2, 0);
|
verifyQueueNumRunnable("queue1.sub2", 2, 0);
|
||||||
|
@ -2333,7 +2324,7 @@ public class TestFairScheduler {
|
||||||
|
|
||||||
// Now test removal of a non-runnable app
|
// Now test removal of a non-runnable app
|
||||||
AppAttemptRemovedSchedulerEvent appRemovedEvent2 =
|
AppAttemptRemovedSchedulerEvent appRemovedEvent2 =
|
||||||
new AppAttemptRemovedSchedulerEvent(attId5, RMAppAttemptState.KILLED);
|
new AppAttemptRemovedSchedulerEvent(attId5, RMAppAttemptState.KILLED, true);
|
||||||
scheduler.handle(appRemovedEvent2);
|
scheduler.handle(appRemovedEvent2);
|
||||||
assertEquals(0, scheduler.maxRunningEnforcer.usersNonRunnableApps
|
assertEquals(0, scheduler.maxRunningEnforcer.usersNonRunnableApps
|
||||||
.get("user1").size());
|
.get("user1").size());
|
||||||
|
@ -2341,7 +2332,7 @@ public class TestFairScheduler {
|
||||||
verifyQueueNumRunnable("queue1.sub3", 0, 0);
|
verifyQueueNumRunnable("queue1.sub3", 0, 0);
|
||||||
// verify it doesn't become runnable when there would be space for it
|
// verify it doesn't become runnable when there would be space for it
|
||||||
AppAttemptRemovedSchedulerEvent appRemovedEvent3 =
|
AppAttemptRemovedSchedulerEvent appRemovedEvent3 =
|
||||||
new AppAttemptRemovedSchedulerEvent(attId4, RMAppAttemptState.FINISHED);
|
new AppAttemptRemovedSchedulerEvent(attId4, RMAppAttemptState.FINISHED, true);
|
||||||
scheduler.handle(appRemovedEvent3);
|
scheduler.handle(appRemovedEvent3);
|
||||||
verifyQueueNumRunnable("queue1.sub2", 1, 0);
|
verifyQueueNumRunnable("queue1.sub2", 1, 0);
|
||||||
verifyQueueNumRunnable("queue1.sub3", 0, 0);
|
verifyQueueNumRunnable("queue1.sub3", 0, 0);
|
||||||
|
@ -2378,7 +2369,7 @@ public class TestFairScheduler {
|
||||||
ApplicationAttemptId appAttemptId =
|
ApplicationAttemptId appAttemptId =
|
||||||
createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
|
createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
|
||||||
fs.addApplication(appAttemptId.getApplicationId(), "queue11", "user11");
|
fs.addApplication(appAttemptId.getApplicationId(), "queue11", "user11");
|
||||||
fs.addApplicationAttempt(appAttemptId);
|
fs.addApplicationAttempt(appAttemptId, false);
|
||||||
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);
|
||||||
|
@ -2389,7 +2380,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.appAttempts.get(appAttemptId);
|
FSSchedulerApp app = fs.getSchedulerApp(appAttemptId);
|
||||||
// Wait until app gets resources.
|
// Wait until app gets resources.
|
||||||
while (app.getCurrentConsumption().equals(Resources.none())) { }
|
while (app.getCurrentConsumption().equals(Resources.none())) { }
|
||||||
|
|
||||||
|
@ -2477,7 +2468,7 @@ public class TestFairScheduler {
|
||||||
|
|
||||||
ApplicationAttemptId appAttemptId =
|
ApplicationAttemptId appAttemptId =
|
||||||
createSchedulingRequest(GB, "root.default", "user", 1);
|
createSchedulingRequest(GB, "root.default", "user", 1);
|
||||||
FSSchedulerApp app = scheduler.appAttempts.get(appAttemptId);
|
FSSchedulerApp app = scheduler.getSchedulerApp(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(),
|
||||||
|
@ -2487,7 +2478,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.appAttempts.get(appAttemptId).isBlacklisted(host));
|
assertFalse(scheduler.getSchedulerApp(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));
|
||||||
|
|
|
@ -156,7 +156,7 @@ public class TestFifoScheduler {
|
||||||
SchedulerEvent appEvent = new AppAddedSchedulerEvent(appId, "queue", "user");
|
SchedulerEvent appEvent = new AppAddedSchedulerEvent(appId, "queue", "user");
|
||||||
schedular.handle(appEvent);
|
schedular.handle(appEvent);
|
||||||
SchedulerEvent attemptEvent =
|
SchedulerEvent attemptEvent =
|
||||||
new AppAttemptAddedSchedulerEvent(appAttemptId);
|
new AppAttemptAddedSchedulerEvent(appAttemptId, false);
|
||||||
schedular.handle(attemptEvent);
|
schedular.handle(attemptEvent);
|
||||||
|
|
||||||
appAttemptId = BuilderUtils.newApplicationAttemptId(appId, 2);
|
appAttemptId = BuilderUtils.newApplicationAttemptId(appId, 2);
|
||||||
|
@ -166,7 +166,7 @@ public class TestFifoScheduler {
|
||||||
"user");
|
"user");
|
||||||
schedular.handle(appEvent2);
|
schedular.handle(appEvent2);
|
||||||
SchedulerEvent attemptEvent2 =
|
SchedulerEvent attemptEvent2 =
|
||||||
new AppAttemptAddedSchedulerEvent(appAttemptId);
|
new AppAttemptAddedSchedulerEvent(appAttemptId, false);
|
||||||
schedular.handle(attemptEvent2);
|
schedular.handle(attemptEvent2);
|
||||||
|
|
||||||
int afterAppsSubmitted = metrics.getAppsSubmitted();
|
int afterAppsSubmitted = metrics.getAppsSubmitted();
|
||||||
|
@ -203,7 +203,7 @@ public class TestFifoScheduler {
|
||||||
"user1");
|
"user1");
|
||||||
scheduler.handle(appEvent);
|
scheduler.handle(appEvent);
|
||||||
AppAttemptAddedSchedulerEvent attemptEvent =
|
AppAttemptAddedSchedulerEvent attemptEvent =
|
||||||
new AppAttemptAddedSchedulerEvent(appAttemptId);
|
new AppAttemptAddedSchedulerEvent(appAttemptId, false);
|
||||||
scheduler.handle(attemptEvent);
|
scheduler.handle(attemptEvent);
|
||||||
|
|
||||||
int memory = 64;
|
int memory = 64;
|
||||||
|
@ -293,7 +293,7 @@ public class TestFifoScheduler {
|
||||||
"user1");
|
"user1");
|
||||||
scheduler.handle(appEvent);
|
scheduler.handle(appEvent);
|
||||||
AppAttemptAddedSchedulerEvent attemptEvent =
|
AppAttemptAddedSchedulerEvent attemptEvent =
|
||||||
new AppAttemptAddedSchedulerEvent(appAttemptId);
|
new AppAttemptAddedSchedulerEvent(appAttemptId, false);
|
||||||
scheduler.handle(attemptEvent);
|
scheduler.handle(attemptEvent);
|
||||||
|
|
||||||
int memory = 1024;
|
int memory = 1024;
|
||||||
|
@ -534,13 +534,6 @@ public class TestFifoScheduler {
|
||||||
LOG.info("--- END: testFifoScheduler ---");
|
LOG.info("--- END: testFifoScheduler ---");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testConcurrentAccessOnApplications() throws Exception {
|
|
||||||
FifoScheduler fs = new FifoScheduler();
|
|
||||||
TestCapacityScheduler.verifyConcurrentAccessOnApplications(
|
|
||||||
fs.appAttempts, FiCaSchedulerApp.class, Queue.class);
|
|
||||||
}
|
|
||||||
|
|
||||||
@SuppressWarnings("resource")
|
@SuppressWarnings("resource")
|
||||||
@Test
|
@Test
|
||||||
public void testBlackListNodes() throws Exception {
|
public void testBlackListNodes() throws Exception {
|
||||||
|
@ -564,18 +557,18 @@ public class TestFifoScheduler {
|
||||||
"user");
|
"user");
|
||||||
fs.handle(appEvent);
|
fs.handle(appEvent);
|
||||||
SchedulerEvent attemptEvent =
|
SchedulerEvent attemptEvent =
|
||||||
new AppAttemptAddedSchedulerEvent(appAttemptId);
|
new AppAttemptAddedSchedulerEvent(appAttemptId, false);
|
||||||
fs.handle(attemptEvent);
|
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(),
|
||||||
Collections.<ContainerId>emptyList(),
|
Collections.<ContainerId>emptyList(),
|
||||||
Collections.singletonList(host), null);
|
Collections.singletonList(host), null);
|
||||||
Assert.assertTrue(fs.getApplication(appAttemptId).isBlacklisted(host));
|
Assert.assertTrue(fs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
|
||||||
fs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
fs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
|
||||||
Collections.<ContainerId>emptyList(), null,
|
Collections.<ContainerId>emptyList(), null,
|
||||||
Collections.singletonList(host));
|
Collections.singletonList(host));
|
||||||
Assert.assertFalse(fs.getApplication(appAttemptId).isBlacklisted(host));
|
Assert.assertFalse(fs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
|
||||||
rm.stop();
|
rm.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1390,7 +1390,7 @@ public class TestRMWebServicesApps extends JerseyTest {
|
||||||
@Test
|
@Test
|
||||||
public void testMultipleAppAttempts() throws JSONException, Exception {
|
public void testMultipleAppAttempts() throws JSONException, Exception {
|
||||||
rm.start();
|
rm.start();
|
||||||
MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
|
MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 8192);
|
||||||
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(),
|
rm.waitForState(app1.getCurrentAppAttempt().getAppAttemptId(),
|
||||||
|
@ -1403,11 +1403,13 @@ public class TestRMWebServicesApps extends JerseyTest {
|
||||||
while (--retriesLeft > 0) {
|
while (--retriesLeft > 0) {
|
||||||
RMAppEvent event =
|
RMAppEvent event =
|
||||||
new RMAppFailedAttemptEvent(app1.getApplicationId(),
|
new RMAppFailedAttemptEvent(app1.getApplicationId(),
|
||||||
RMAppEventType.ATTEMPT_FAILED, "");
|
RMAppEventType.ATTEMPT_FAILED, "", false);
|
||||||
app1.handle(event);
|
app1.handle(event);
|
||||||
rm.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
|
rm.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);
|
||||||
amNodeManager.nodeHeartbeat(true);
|
amNodeManager.nodeHeartbeat(true);
|
||||||
}
|
}
|
||||||
|
// kick the scheduler to allocate the am container.
|
||||||
|
amNodeManager.nodeHeartbeat(true);
|
||||||
rm.waitForState(app1.getCurrentAppAttempt().getAppAttemptId(),
|
rm.waitForState(app1.getCurrentAppAttempt().getAppAttemptId(),
|
||||||
RMAppAttemptState.ALLOCATED);
|
RMAppAttemptState.ALLOCATED);
|
||||||
assertEquals("incorrect number of attempts", maxAppAttempts,
|
assertEquals("incorrect number of attempts", maxAppAttempts,
|
||||||
|
|
Loading…
Reference in New Issue