Merge -r 1169979:1169980 from trunk to branch-0.23 to fix MAPREDUCE-2896.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1169981 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Arun Murthy 2011-09-13 00:07:09 +00:00
parent 6ad35c6378
commit 677c212467
70 changed files with 1264 additions and 588 deletions

View File

@ -256,6 +256,10 @@ Release 0.23.0 - Unreleased
MAPREDUCE-2675. Reformat JobHistory Server main page to be more
useful. (Robert Joseph Evans via vinodkv).
MAPREDUCE-2896. Simplify all apis to in
org.apache.hadoop.yarn.api.records.* to be get/set only. Added javadocs to
all public records. (acmurthy)
OPTIMIZATIONS
MAPREDUCE-2026. Make JobTracker.getJobCounters() and

View File

@ -137,6 +137,7 @@ public abstract class TaskAttemptImpl implements
protected final Configuration conf;
protected final Path jobFile;
protected final int partition;
@SuppressWarnings("rawtypes")
protected final EventHandler eventHandler;
private final TaskAttemptId attemptId;
private final Clock clock;
@ -431,7 +432,8 @@ TaskAttemptEventType.TA_CONTAINER_CLEANED, new TaskCleanupTransition())
//this is the last status reported by the REMOTE running attempt
private TaskAttemptStatus reportedStatus;
public TaskAttemptImpl(TaskId taskId, int i, EventHandler eventHandler,
public TaskAttemptImpl(TaskId taskId, int i,
@SuppressWarnings("rawtypes") EventHandler eventHandler,
TaskAttemptListener taskAttemptListener, Path jobFile, int partition,
Configuration conf, String[] dataLocalHosts, OutputCommitter committer,
Token<JobTokenIdentifier> jobToken,
@ -527,6 +529,13 @@ private ContainerLaunchContext createContainerLaunchContext() {
ContainerLaunchContext container =
recordFactory.newRecordInstance(ContainerLaunchContext.class);
// Application resources
Map<String, LocalResource> localResources =
new HashMap<String, LocalResource>();
// Application environment
Map<String, String> environment = new HashMap<String, String>();
try {
FileSystem remoteFS = FileSystem.get(conf);
@ -535,7 +544,7 @@ private ContainerLaunchContext createContainerLaunchContext() {
Path remoteJobJar = (new Path(remoteTask.getConf().get(
MRJobConfig.JAR))).makeQualified(remoteFS.getUri(),
remoteFS.getWorkingDirectory());
container.setLocalResource(
localResources.put(
MRConstants.JOB_JAR,
createLocalResource(remoteFS, recordFactory, remoteJobJar,
LocalResourceType.FILE, LocalResourceVisibility.APPLICATION));
@ -557,7 +566,7 @@ private ContainerLaunchContext createContainerLaunchContext() {
new Path(path, oldJobId.toString());
Path remoteJobConfPath =
new Path(remoteJobSubmitDir, MRConstants.JOB_CONF_FILE);
container.setLocalResource(
localResources.put(
MRConstants.JOB_CONF_FILE,
createLocalResource(remoteFS, recordFactory, remoteJobConfPath,
LocalResourceType.FILE, LocalResourceVisibility.APPLICATION));
@ -565,9 +574,14 @@ private ContainerLaunchContext createContainerLaunchContext() {
+ remoteJobConfPath.toUri().toASCIIString());
// //////////// End of JobConf setup
// Setup DistributedCache
setupDistributedCache(remoteFS, conf, container);
setupDistributedCache(remoteFS, conf, localResources, environment);
// Set local-resources and environment
container.setLocalResources(localResources);
container.setEnv(environment);
// Setup up tokens
Credentials taskCredentials = new Credentials();
@ -594,12 +608,12 @@ private ContainerLaunchContext createContainerLaunchContext() {
// Add shuffle token
LOG.info("Putting shuffle token in serviceData");
container
.setServiceData(
ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID,
ShuffleHandler.serializeServiceData(jobToken));
Map<String, ByteBuffer> serviceData = new HashMap<String, ByteBuffer>();
serviceData.put(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID,
ShuffleHandler.serializeServiceData(jobToken));
container.setServiceData(serviceData);
MRApps.addToClassPath(container.getAllEnv(), getInitialClasspath());
MRApps.addToClassPath(container.getEnv(), getInitialClasspath());
} catch (IOException e) {
throw new YarnException(e);
}
@ -622,11 +636,11 @@ private ContainerLaunchContext createContainerLaunchContext() {
classPaths.add(workDir.toString()); // TODO
// Construct the actual Container
container.addAllCommands(MapReduceChildJVM.getVMCommand(
container.setCommands(MapReduceChildJVM.getVMCommand(
taskAttemptListener.getAddress(), remoteTask, javaHome,
workDir.toString(), containerLogDir, childTmpDir, jvmID));
MapReduceChildJVM.setVMEnv(container.getAllEnv(), classPaths,
MapReduceChildJVM.setVMEnv(container.getEnv(), classPaths,
workDir.toString(), containerLogDir, nmLdLibraryPath, remoteTask,
localizedApplicationTokensFile);
@ -648,11 +662,15 @@ private static long[] parseTimeStamps(String[] strs) {
return result;
}
private void setupDistributedCache(FileSystem remoteFS, Configuration conf,
ContainerLaunchContext container) throws IOException {
private void setupDistributedCache(FileSystem remoteFS,
Configuration conf,
Map<String, LocalResource> localResources,
Map<String, String> env)
throws IOException {
// Cache archives
parseDistributedCacheArtifacts(remoteFS, container, LocalResourceType.ARCHIVE,
parseDistributedCacheArtifacts(remoteFS, localResources, env,
LocalResourceType.ARCHIVE,
DistributedCache.getCacheArchives(conf),
parseTimeStamps(DistributedCache.getArchiveTimestamps(conf)),
getFileSizes(conf, MRJobConfig.CACHE_ARCHIVES_SIZES),
@ -660,7 +678,9 @@ private void setupDistributedCache(FileSystem remoteFS, Configuration conf,
DistributedCache.getArchiveClassPaths(conf));
// Cache files
parseDistributedCacheArtifacts(remoteFS, container, LocalResourceType.FILE,
parseDistributedCacheArtifacts(remoteFS,
localResources, env,
LocalResourceType.FILE,
DistributedCache.getCacheFiles(conf),
parseTimeStamps(DistributedCache.getFileTimestamps(conf)),
getFileSizes(conf, MRJobConfig.CACHE_FILES_SIZES),
@ -672,7 +692,10 @@ private void setupDistributedCache(FileSystem remoteFS, Configuration conf,
// Use TaskDistributedCacheManager.CacheFiles.makeCacheFiles(URI[],
// long[], boolean[], Path[], FileType)
private void parseDistributedCacheArtifacts(
FileSystem remoteFS, ContainerLaunchContext container, LocalResourceType type,
FileSystem remoteFS,
Map<String, LocalResource> localResources,
Map<String, String> env,
LocalResourceType type,
URI[] uris, long[] timestamps, long[] sizes, boolean visibilities[],
Path[] pathsToPutOnClasspath) throws IOException {
@ -709,7 +732,7 @@ private void parseDistributedCacheArtifacts(
throw new IllegalArgumentException("Resource name must be relative");
}
String linkName = name.toUri().getPath();
container.setLocalResource(
localResources.put(
linkName,
BuilderUtils.newLocalResource(
p.toUri(), type,
@ -719,8 +742,7 @@ private void parseDistributedCacheArtifacts(
sizes[i], timestamps[i])
);
if (classPaths.containsKey(u.getPath())) {
Map<String, String> environment = container.getAllEnv();
MRApps.addToClassPath(environment, linkName);
MRApps.addToClassPath(env, linkName);
}
}
}
@ -892,6 +914,7 @@ public TaskAttemptState getState() {
}
}
@SuppressWarnings("unchecked")
@Override
public void handle(TaskAttemptEvent event) {
LOG.info("Processing " + event.getTaskAttemptID() +
@ -1034,6 +1057,7 @@ private static class RequestContainerTransition implements
public RequestContainerTransition(boolean rescheduled) {
this.rescheduled = rescheduled;
}
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt,
TaskAttemptEvent event) {
@ -1062,6 +1086,7 @@ public void transition(TaskAttemptImpl taskAttempt,
private static class ContainerAssignedTransition implements
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
@SuppressWarnings("unchecked")
@Override
public void transition(final TaskAttemptImpl taskAttempt,
TaskAttemptEvent event) {
@ -1111,6 +1136,7 @@ private static class DeallocateContainerTransition implements
this.finalState = finalState;
this.withdrawsContainerRequest = withdrawsContainerRequest;
}
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt,
TaskAttemptEvent event) {
@ -1157,6 +1183,7 @@ public void transition(TaskAttemptImpl taskAttempt,
private static class LaunchedContainerTransition implements
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt,
TaskAttemptEvent evnt) {
@ -1207,6 +1234,7 @@ public void transition(TaskAttemptImpl taskAttempt,
private static class CommitPendingTransition implements
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt,
TaskAttemptEvent event) {
@ -1218,6 +1246,7 @@ public void transition(TaskAttemptImpl taskAttempt,
private static class TaskCleanupTransition implements
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt,
TaskAttemptEvent event) {
@ -1233,6 +1262,7 @@ public void transition(TaskAttemptImpl taskAttempt,
private static class SucceededTransition implements
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt,
TaskAttemptEvent event) {
@ -1262,6 +1292,7 @@ public void transition(TaskAttemptImpl taskAttempt,
private static class FailedTransition implements
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) {
// set the finish time
@ -1286,6 +1317,7 @@ public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) {
}
}
@SuppressWarnings({ "unchecked" })
private void logAttemptFinishedEvent(TaskAttemptState state) {
//Log finished events only if an attempt started.
if (getLaunchTime() == 0) return;
@ -1319,6 +1351,7 @@ private void logAttemptFinishedEvent(TaskAttemptState state) {
private static class TooManyFetchFailureTransition implements
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) {
//add to diagnostic
@ -1346,6 +1379,7 @@ public void transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent event) {
private static class KilledTransition implements
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt,
TaskAttemptEvent event) {
@ -1372,6 +1406,7 @@ public void transition(TaskAttemptImpl taskAttempt,
private static class CleanupContainerTransition implements
SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt,
TaskAttemptEvent event) {
@ -1398,6 +1433,7 @@ private void addDiagnosticInfo(String diag) {
private static class StatusUpdater
implements SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
@SuppressWarnings("unchecked")
@Override
public void transition(TaskAttemptImpl taskAttempt,
TaskAttemptEvent event) {

View File

@ -70,7 +70,7 @@ public void handle(ContainerAllocatorEvent event) {
if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
LOG.info("Processing the event " + event.toString());
ContainerId cID = recordFactory.newRecordInstance(ContainerId.class);
cID.setAppId(appID);
cID.setApplicationAttemptId(applicationAttemptId);
// use negative ids to denote that these are local. Need a better way ??
cID.setId((-1) * containerCount.getAndIncrement());

View File

@ -63,7 +63,6 @@
import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleanupEvent;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -324,7 +323,7 @@ protected ContainerAllocator createContainerAllocator(
@Override
public void handle(ContainerAllocatorEvent event) {
ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
cId.setAppId(getContext().getApplicationID());
cId.setApplicationAttemptId(getContext().getApplicationAttemptId());
cId.setId(containerCount++);
Container container = recordFactory.newRecordInstance(Container.class);
container.setId(cId);

View File

@ -30,6 +30,7 @@
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
@ -124,12 +125,15 @@ public void run() {
try {
if (concurrentRunningTasks < maxConcurrentRunningTasks) {
event = eventQueue.take();
ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
cId.setAppId(getContext().getApplicationID());
ContainerId cId =
recordFactory.newRecordInstance(ContainerId.class);
cId.setApplicationAttemptId(
getContext().getApplicationAttemptId());
cId.setId(containerCount++);
//System.out.println("Allocating " + containerCount);
Container container = recordFactory.newRecordInstance(Container.class);
Container container =
recordFactory.newRecordInstance(Container.class);
container.setId(cId);
NodeId nodeId = recordFactory.newRecordInstance(NodeId.class);
nodeId.setHost("dummy");

View File

@ -51,6 +51,7 @@
import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.MockApps;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.util.Records;
@ -235,7 +236,11 @@ public boolean isFinished() {
@Override
public ContainerId getAssignedContainerID() {
ContainerId id = Records.newRecord(ContainerId.class);
id.setAppId(taid.getTaskId().getJobId().getAppId());
ApplicationAttemptId appAttemptId =
Records.newRecord(ApplicationAttemptId.class);
appAttemptId.setApplicationId(taid.getTaskId().getJobId().getAppId());
appAttemptId.setAttemptId(0);
id.setApplicationAttemptId(appAttemptId);
return id;
}

View File

@ -29,6 +29,7 @@
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@ -82,12 +83,23 @@ public class CompletedTaskAttempt implements TaskAttempt {
@Override
public ContainerId getAssignedContainerID() {
//TODO ContainerId needs to be part of some historyEvent to be able to render the log directory.
ContainerId containerId = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ContainerId.class);
//TODO ContainerId needs to be part of some historyEvent to be able to
//render the log directory.
ContainerId containerId =
RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
ContainerId.class);
containerId.setId(-1);
containerId.setAppId(RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class));
containerId.getAppId().setId(-1);
containerId.getAppId().setClusterTimestamp(-1);
ApplicationAttemptId applicationAttemptId =
RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
ApplicationAttemptId.class);
applicationAttemptId.setAttemptId(-1);
ApplicationId applicationId =
RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
ApplicationId.class);
applicationId.setClusterTimestamp(-1);
applicationId.setId(-1);
applicationAttemptId.setApplicationId(applicationId);
containerId.setApplicationAttemptId(applicationAttemptId);
return containerId;
}

View File

@ -348,7 +348,6 @@ public ApplicationSubmissionContext createApplicationSubmissionContext(
// Add { job jar, MR app jar } to classpath.
Map<String, String> environment = new HashMap<String, String>();
// appContext.environment = new HashMap<CharSequence, CharSequence>();
MRApps.setInitialClasspath(environment);
MRApps.addToClassPath(environment, MRConstants.JOB_JAR);
MRApps.addToClassPath(environment,

View File

@ -119,11 +119,7 @@ public interface AMResponse {
@Stable
public List<ContainerStatus> getCompletedContainersStatuses();
/**
* Set the list of list of <em>completed containers' statuses</em>.
* @param containers list of <em>completed containers' statuses</em>
*/
@Public
@Stable
@Private
@Unstable
public void setCompletedContainersStatuses(List<ContainerStatus> containers);
}

View File

@ -20,12 +20,44 @@
import java.text.NumberFormat;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
* <p><code>ApplicationAttemptId</code> denotes the particular <em>attempt</em>
* of an <code>ApplicationMaster</code> for a given {@link ApplicationId}.</p>
*
* <p>Multiple attempts might be needed to run an application to completion due
* to temporal failures of the <code>ApplicationMaster</code> such as hardware
* failures, connectivity issues etc. on the node on which it was scheduled.</p>
*/
@Public
@Stable
public abstract class ApplicationAttemptId implements
Comparable<ApplicationAttemptId> {
/**
* Get the <code>ApplicationId</code> of the <code>ApplicationAttempId</code>.
* @return <code>ApplicationId</code> of the <code>ApplicationAttempId</code>
*/
@Public
@Stable
public abstract ApplicationId getApplicationId();
@Private
@Unstable
public abstract void setApplicationId(ApplicationId appID);
/**
* Get the <code>attempt id</code> of the <code>Application</code>.
* @return <code>attempt id</code> of the <code>Application</code>
*/
public abstract int getAttemptId();
public abstract void setApplicationId(ApplicationId appID);
@Private
@Unstable
public abstract void setAttemptId(int attemptId);

View File

@ -18,11 +18,47 @@
package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
* <p><code>ApplicationId</code> represents the <em>globally unique</em>
* identifier for an application.</p>
*
* <p>The globally unique nature of the identifier is achieved by using the
* <em>cluster timestamp</em> i.e. start-time of the
* <code>ResourceManager</code> along with a monotonically increasing counter
* for the application.</p>
*/
@Public
@Stable
public abstract class ApplicationId implements Comparable<ApplicationId> {
/**
* Get the short integer identifier of the <code>ApplicationId</code>
* which is unique for all applications started by a particular instance
* of the <code>ResourceManager</code>.
* @return short integer identifier of the <code>ApplicationId</code>
*/
@Public
@Stable
public abstract int getId();
@Private
@Unstable
public abstract void setId(int id);
/**
* Get the <em>start time</em> of the <code>ResourceManager</code> which is
* used to generate globally unique <code>ApplicationId</code>.
* @return <em>start time</em> of the <code>ResourceManager</code>
*/
public abstract long getClusterTimestamp();
public abstract void setId(int id);
@Private
@Unstable
public abstract void setClusterTimestamp(long clusterTimestamp);

View File

@ -18,26 +18,43 @@
package org.apache.hadoop.yarn.api.records;
//TODO: Split separate object for register, deregister and in-RM use.
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
* <em>For internal use only...</em>
*/
@Private
@Unstable
public interface ApplicationMaster {
ApplicationId getApplicationId();
String getHost();
int getRpcPort();
String getTrackingUrl();
ApplicationStatus getStatus();
ApplicationState getState();
String getClientToken();
int getAMFailCount();
int getContainerCount();
String getDiagnostics();
void setApplicationId(ApplicationId appId);
String getHost();
void setHost(String host);
int getRpcPort();
void setRpcPort(int rpcPort);
String getTrackingUrl();
void setTrackingUrl(String url);
ApplicationStatus getStatus();
void setStatus(ApplicationStatus status);
ApplicationState getState();
void setState(ApplicationState state);
String getClientToken();
void setClientToken(String clientToken);
int getAMFailCount();
void setAMFailCount(int amFailCount);
int getContainerCount();
void setContainerCount(int containerCount);
String getDiagnostics();
void setDiagnostics(String diagnostics);
}

View File

@ -18,6 +18,30 @@
package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
/**
* Ennumeration of various states of an <code>Application</code>.
*/
@Public
@Stable
public enum ApplicationState {
NEW, SUBMITTED, RUNNING, SUCCEEDED, FAILED, KILLED
/** Application which was just created. */
NEW,
/** Application which has been submitted. */
SUBMITTED,
/** Application which is currently running. */
RUNNING,
/** Application which completed successfully. */
SUCCEEDED,
/** Application which failed. */
FAILED,
/** Application which was terminated by a user or admin. */
KILLED
}

View File

@ -18,12 +18,21 @@
package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
* <em>For internal use only...</em>
*/
@Private
@Unstable
public interface ApplicationStatus {
ApplicationAttemptId getApplicationAttemptId();
int getResponseId();
float getProgress();
void setApplicationAttemptId(ApplicationAttemptId applicationAttemptId);
int getResponseId();
void setResponseId(int id);
float getProgress();
void setProgress(float progress);
}

View File

@ -18,21 +18,133 @@
package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.AMRMProtocol;
import org.apache.hadoop.yarn.api.ContainerManager;
/**
* <p><code>Container</code> represents an allocated resource in the cluster.
* </p>
*
* <p>The <code>ResourceManager</code> is the sole authority to allocate any
* <code>Container</code> to applications. The allocated <code>Container</code>
* is always on a single node and has a unique {@link ContainerId}. It has
* a specific amount of {@link Resource} allocated.</p>
*
* <p>It includes details such as:
* <ul>
* <li>{@link ContainerId} for the container, which is globally unique.</li>
* <li>
* {@link NodeId} of the node on which identifies the node on which it
* is allocated.
* </li>
* <li>HTTP uri of the node.</li>
* <li>{@link Resource} allocated to the container.</li>
* <li>{@link ContainerState} of the container.</li>
* <li>
* {@link ContainerToken} of the container, used to securely verify
* authenticity of the allocation.
* </li>
* <li>{@link ContainerStatus} of the container.</li>
* </ul>
* </p>
*
* <p>Typically, an <code>ApplicationMaster</code> receives the
* <code>Container</code> from the <code>ResourceManager</code> during
* resource-negotiation and then talks to the <code>NodManager</code> to
* start/stop containers.</p>
*
* @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
* @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
* @see ContainerManager#stopContainer(org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest)
*/
@Public
@Stable
public interface Container extends Comparable<Container> {
/**
* Get the globally unique identifier for the container.
* @return globally unique identifier for the container
*/
@Public
@Stable
ContainerId getId();
@Private
@Unstable
void setId(ContainerId id);
/**
* Get the identifier of the node on which the container is allocated.
* @return identifier of the node on which the container is allocated
*/
@Public
@Stable
NodeId getNodeId();
@Private
@Unstable
void setNodeId(NodeId nodeId);
/**
* Get the http uri of the node on which the container is allocated.
* @return http uri of the node on which the container is allocated
*/
@Public
@Stable
String getNodeHttpAddress();
@Private
@Unstable
void setNodeHttpAddress(String nodeHttpAddress);
/**
* Get the <code>Resource</code> allocated to the container.
* @return <code>Resource</code> allocated to the container
*/
@Public
@Stable
Resource getResource();
@Private
@Unstable
void setResource(Resource resource);
/**
* Get the current <code>ContainerState</code> of the container.
* @return current <code>ContainerState</code> of the container
*/
@Public
@Stable
ContainerState getState();
@Private
@Unstable
void setState(ContainerState state);
/**
* Get the <code>ContainerToken</code> for the container.
* @return <code>ContainerToken</code> for the container
*/
@Public
@Stable
ContainerToken getContainerToken();
@Private
@Unstable
void setContainerToken(ContainerToken containerToken);
/**
* Get the <code>ContainerStatus</code> of the container.
* @return <code>ContainerStatus</code> of the container
*/
@Public
@Stable
ContainerStatus getContainerStatus();
void setId(ContainerId id);
void setNodeId(NodeId nodeId);
void setNodeHttpAddress(String nodeHttpAddress);
void setResource(Resource resource);
void setState(ContainerState state);
void setContainerToken(ContainerToken containerToken);
@Private
@Unstable
void setContainerStatus(ContainerStatus containerStatus);
}

View File

@ -20,13 +20,42 @@
import java.text.NumberFormat;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
* <p><code>ContainerId</code> represents a globally unique identifier
* for a {@link Container} in the cluster.</p>
*/
@Public
@Stable
public abstract class ContainerId implements Comparable<ContainerId>{
public abstract ApplicationAttemptId getAppAttemptId();
public abstract ApplicationId getAppId();
public abstract int getId();
/**
* Get the <code>ApplicationAttemptId</code> of the application to which
* the <code>Container</code> was assigned.
* @return <code>ApplicationAttemptId</code> of the application to which
* the <code>Container</code> was assigned
*/
@Public
@Stable
public abstract ApplicationAttemptId getApplicationAttemptId();
public abstract void setAppAttemptId(ApplicationAttemptId atId);
public abstract void setAppId(ApplicationId appID);
@Private
@Unstable
public abstract void setApplicationAttemptId(ApplicationAttemptId atId);
/**
* Get the identifier of the <code>ContainerId</code>.
* @return identifier of the <code>ContainerId</code>
*/
@Public
@Stable
public abstract int getId();
@Private
@Unstable
public abstract void setId(int id);
@ -74,7 +103,7 @@ public int hashCode() {
int result = 1;
result = prime * result + getId();
result = prime * result
+ ((getAppAttemptId() == null) ? 0 : getAppAttemptId().hashCode());
+ ((getApplicationAttemptId() == null) ? 0 : getApplicationAttemptId().hashCode());
return result;
}
@ -85,7 +114,8 @@ public boolean equals(Object other) {
}
if (other.getClass().isAssignableFrom(this.getClass())) {
ContainerId otherCId = (ContainerId)other;
if (this.getAppAttemptId().equals(otherCId.getAppAttemptId())) {
if (this.getApplicationAttemptId().equals(
otherCId.getApplicationAttemptId())) {
return this.getId() == otherCId.getId();
}
}
@ -94,10 +124,12 @@ public boolean equals(Object other) {
@Override
public int compareTo(ContainerId other) {
if (this.getAppAttemptId().compareTo(other.getAppAttemptId()) == 0) {
if (this.getApplicationAttemptId().compareTo(
other.getApplicationAttemptId()) == 0) {
return this.getId() - other.getId();
} else {
return this.getAppAttemptId().compareTo(other.getAppAttemptId());
return this.getApplicationAttemptId().compareTo(
other.getApplicationAttemptId());
}
}
@ -105,10 +137,10 @@ public int compareTo(ContainerId other) {
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
ApplicationId appId = getAppId();
ApplicationId appId = getApplicationAttemptId().getApplicationId();
sb.append("container_").append(appId.getClusterTimestamp()).append("_");
sb.append(appIdFormat.get().format(appId.getId())).append("_");
sb.append(appAttemptIdFormat.get().format(getAppAttemptId().
sb.append(appAttemptIdFormat.get().format(getApplicationAttemptId().
getAttemptId())).append("_");
sb.append(containerIdFormat.get().format(getId()));
return sb.toString();

View File

@ -22,10 +22,8 @@
import java.util.List;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.ContainerManager;
/**
@ -121,100 +119,52 @@ public interface ContainerLaunchContext {
void setContainerTokens(ByteBuffer containerToken);
/**
* Get all <code>LocalResource</code> required by the container.
* Get <code>LocalResource</code> required by the container.
* @return all <code>LocalResource</code> required by the container
*/
@Public
@Stable
Map<String, LocalResource> getAllLocalResources();
Map<String, LocalResource> getLocalResources();
@Private
@Unstable
LocalResource getLocalResource(String key);
/**
* Add all <code>LocalResource</code> required by the container.
* Set <code>LocalResource</code> required by the container.
* @param localResources <code>LocalResource</code> required by the container
*/
@Public
@Stable
void addAllLocalResources(Map<String, LocalResource> localResources);
@Private
@Unstable
void setLocalResource(String key, LocalResource value);
@Private
@Unstable
void removeLocalResource(String key);
@Private
@Unstable
void clearLocalResources();
void setLocalResources(Map<String, LocalResource> localResources);
/**
* Get application-specific binary service data.
* @return application-specific binary service data
* Get application-specific binary <em>service data</em>.
* @return application-specific binary <em>service data</em>
*/
@Public
@Stable
Map<String, ByteBuffer> getAllServiceData();
@Private
@Unstable
ByteBuffer getServiceData(String key);
/**
* Add add application-specific binary service data.
* @param serviceData application-specific binary service data
*/
@Public
@Stable
void addAllServiceData(Map<String, ByteBuffer> serviceData);
@Private
@Unstable
void setServiceData(String key, ByteBuffer value);
@Private
@Unstable
void removeServiceData(String key);
@Private
@Unstable
void clearServiceData();
/**
* Get <em>environment variables</em> for the launched container.
* @return <em>environment variables</em> for the launched container
*/
@Public
@Stable
Map<String, String> getAllEnv();
@Private
@Unstable
String getEnv(String key);
Map<String, ByteBuffer> getServiceData();
/**
* Add <em>environment variables</em> for the launched container.
* @param env <em>environment variables</em> for the launched container
* Set application-specific binary <em>service data</em>.
* @param serviceData application-specific binary <em>service data</em>
*/
@Public
@Stable
void addAllEnv(Map<String, String> env);
void setServiceData(Map<String, ByteBuffer> serviceData);
@Private
@Unstable
void setEnv(String key, String value);
@Private
@Unstable
void removeEnv(String key);
@Private
@Unstable
void clearEnv();
/**
* Get <em>environment variables</em> for the container.
* @return <em>environment variables</em> for the container
*/
@Public
@Stable
Map<String, String> getEnv();
/**
* Add <em>environment variables</em> for the container.
* @param environment <em>environment variables</em> for the container
*/
@Public
@Stable
void setEnv(Map<String, String> environment);
/**
* Get the list of <em>commands</em> for launching the container.
@ -222,15 +172,7 @@ public interface ContainerLaunchContext {
*/
@Public
@Stable
List<String> getCommandList();
@Private
@Unstable
String getCommand(int index);
@Private
@Unstable
int getCommandCount();
List<String> getCommands();
/**
* Add the list of <em>commands</em> for launching the container.
@ -238,17 +180,6 @@ public interface ContainerLaunchContext {
*/
@Public
@Stable
void addAllCommands(List<String> commands);
void setCommands(List<String> commands);
@Private
@Unstable
void addCommand(String command);
@Private
@Unstable
void removeCommand(int index);
@Private
@Unstable
void clearCommands();
}

View File

@ -18,6 +18,16 @@
package org.apache.hadoop.yarn.api.records;
/**
* <p>State of a <code>Container</code>.</p>
*/
public enum ContainerState {
NEW, RUNNING, COMPLETE
/** New container */
NEW,
/** Running container */
RUNNING,
/** Completed container */
COMPLETE
}

View File

@ -18,14 +18,81 @@
package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
* <p><code>ContainerStatus</code> represents the current status of a
* <code>Container</code>.</p>
*
* <p>It provides details such as:
* <ul>
* <li><code>ContainerId</code> of the container.</li>
* <li><code>ContainerState</code> of the container.</li>
* <li><em>Exit status</em> of a completed container.</li>
* <li><em>Diagnostic</em> message for a failed container.</li>
* </ul>
* </p>
*/
@Public
@Stable
public interface ContainerStatus {
/**
* Get the <code>ContainerId</code> of the container.
* @return <code>ContainerId</code> of the container
*/
@Public
@Stable
ContainerId getContainerId();
@Private
@Unstable
void setContainerId(ContainerId containerId);
/**
* Get the <code>ContainerState</code> of the container.
* @return <code>ContainerState</code> of the container
*/
@Public
@Stable
ContainerState getState();
String getExitStatus();
@Private
@Unstable
void setState(ContainerState state);
/**
* <p>Get the <em>exit status</em> for the container.</p>
*
* <p>Note: This is valid only for completed containers i.e. containers
* with state {@link ContainerState#COMPLETE}.
* Otherwise, it returns an invalid exit code equal to {@literal -1000};</p>
*
* <p>Container killed by the framework, either due to being released by
* the application or being 'lost' due to node failures etc. have a special
* exit code of {@literal -100}.</p>
*
* @return <em>exit status</em> for the container
*/
@Public
@Stable
int getExitStatus();
@Private
@Unstable
void setExitStatus(int exitStatus);
/**
* Get <em>diagnostic messages</em> for failed containers.
* @return <em>diagnostic messages</em> for failed containers
*/
@Public
@Stable
String getDiagnostics();
void setContainerId(ContainerId containerId);
void setState(ContainerState state);
void setExitStatus(String exitStatus);
@Private
@Unstable
void setDiagnostics(String diagnostics);
}

View File

@ -20,15 +20,76 @@
import java.nio.ByteBuffer;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.yarn.api.AMRMProtocol;
import org.apache.hadoop.yarn.api.ContainerManager;
/**
* <p><code>ContainerToken</code> is the security token used by the framework
* to verify authenticity of any <code>Container</code>.</p>
*
* <p>The <code>ResourceManager</code>, on container allocation provides a
* secure token which is verified by the <code>NodeManager</code> on
* container launch.</p>
*
* <p>Applications do not need to care about <code>ContainerToken</code>, they
* are transparently handled by the framework - the allocated
* <code>Container</code> includes the <code>ContainerToken</code>.</p>
*
* @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
* @see ContainerManager#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
*/
@Public
@Stable
public interface ContainerToken {
/**
* Get the token identifier.
* @return token identifier
*/
@Public
@Stable
public abstract ByteBuffer getIdentifier();
public abstract ByteBuffer getPassword();
public abstract String getKind();
public abstract String getService();
@Private
@Stable
public abstract void setIdentifier(ByteBuffer identifier);
/**
* Get the token password
* @return token password
*/
@Public
@Stable
public abstract ByteBuffer getPassword();
@Private
@Stable
public abstract void setPassword(ByteBuffer password);
/**
* Get the token kind.
* @return token kind
*/
@Public
@Stable
public abstract String getKind();
@Private
@Stable
public abstract void setKind(String kind);
/**
* Get the service to which the token is allocated.
* @return service to which the token is allocated
*/
@Public
@Stable
public abstract String getService();
@Private
@Stable
public abstract void setService(String service);
}

View File

@ -18,11 +18,43 @@
package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
* <p><code>NodeId</code> is the unique identifier for a node.</p>
*
* <p>It includes the <em>hostname</em> and <em>port</em> to uniquely
* identify the node. Thus, it is unique across restarts of any
* <code>NodeManager</code>.</p>
*/
@Public
@Stable
public interface NodeId extends Comparable<NodeId> {
/**
* Get the <em>hostname</em> of the node.
* @return <em>hostname</em> of the node
*/
@Public
@Stable
String getHost();
@Private
@Unstable
void setHost(String host);
/**
* Get the <em>port</em> for communicating with the node.
* @return <em>port</em> for communicating with the node
*/
@Public
@Stable
int getPort();
@Private
@Unstable
void setPort(int port);
}

View File

@ -20,11 +20,15 @@
import java.nio.ByteBuffer;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.util.ProtoUtils;
import com.google.protobuf.ByteString;
import com.google.protobuf.Message;
@Private
@Unstable
public abstract class ProtoBase <T extends Message> {
public abstract T getProto();

View File

@ -18,10 +18,40 @@
package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.yarn.api.AMRMProtocol;
/**
* <p><code>Resource</code> models a set of computer resources in the
* cluster.</p>
*
* <p>Currrently it only models <em>memory</em>.</p>
*
* <p>Typically, applications request <code>Resource</code> of suitable
* capability to run their component tasks.</p>
*
* @see ResourceRequest
* @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
*/
@Public
@Stable
public interface Resource extends Comparable<Resource> {
/**
* Get <em>memory</em> of the resource.
* @return <em>memory</em> of the resource
*/
@Public
@Stable
public abstract int getMemory();
/**
* Set <em>memory</em> of the resource.
* @param memory <em>memory</em> of the resource
*/
@Public
@Stable
public abstract void setMemory(int memory);
}

View File

@ -18,16 +18,107 @@
package org.apache.hadoop.yarn.api.records;
public interface ResourceRequest extends Comparable<ResourceRequest>{
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.yarn.api.AMRMProtocol;
/**
* <p><code>ResourceRequest</code> represents the request made by an
* application to the <code>ResourceManager</code> to obtain various
* <code>Container</code> allocations.</p>
*
* <p>It includes:
* <ul>
* <li>{@link Priority} of the request.</li>
* <li>
* The <em>name</em> of the machine or rack on which the allocation is
* desired. A special value of <em>*</em> signifies that
* <em>any</em> host/rack is acceptable to the application.
* </li>
* <li>{@link Resource} required for each request.</li>
* <li>
* Number of containers of such specifications which are required
* by the application.
* </li>
* </ul>
* </p>
*
* @see Resource
* @see AMRMProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
*/
@Public
@Stable
public interface ResourceRequest extends Comparable<ResourceRequest> {
/**
* Get the <code>Priority</code> of the request.
* @return <code>Priority</code> of the request
*/
@Public
@Stable
public abstract Priority getPriority();
/**
* Set the <code>Priority</code> of the request
* @param priority <code>Priority</code> of the request
*/
@Public
@Stable
public abstract void setPriority(Priority priority);
/**
* Get the <em>host/rack</em> on which the allocation is desired.
*
* A special value of <em>*</em> signifies that <em>any</em> host/rack is
* acceptable.
*
* @return <em>host/rack</em> on which the allocation is desired
*/
@Public
@Stable
public abstract String getHostName();
/**
* Set <em>host/rack</em> on which the allocation is desired.
*
* A special value of <em>*</em> signifies that <em>any</em> host/rack is
* acceptable.
*
* @param hostName <em>host/rack</em> on which the allocation is desired
*/
@Public
@Stable
public abstract void setHostName(String hostName);
/**
* Get the <code>Resource</code> capability of the request.
* @return <code>Resource</code> capability of the request
*/
@Public
@Stable
public abstract Resource getCapability();
/**
* Set the <code>Resource</code> capability of the request
* @param capability <code>Resource</code> capability of the request
*/
@Public
@Stable
public abstract void setCapability(Resource capability);
/**
* Get the number of containers required with the given specifications.
* @return number of containers required with the given specifications
*/
@Public
@Stable
public abstract int getNumContainers();
public abstract void setPriority(Priority priority);
public abstract void setHostName(String hostName);
public abstract void setCapability(Resource capability);
/**
* Set the number of containers required with the given specifications
* @param numContainers number of containers required with the given
* specifications
*/
@Public
@Stable
public abstract void setNumContainers(int numContainers);
}

View File

@ -18,14 +18,77 @@
package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Evolving;
/**
* <p><code>URL</code> represents a serializable {@link java.net.URL}.</p>
*/
@Public
@Evolving
public interface URL {
/**
* Get the scheme of the URL.
* @return scheme of the URL
*/
@Public
@Evolving
public abstract String getScheme();
/**
* Set the scheme of the URL
* @param scheme scheme of the URL
*/
@Public
@Evolving
public abstract void setScheme(String scheme);
/**
* Get the host of the URL.
* @return host of the URL
*/
@Public
@Evolving
public abstract String getHost();
/**
* Set the host of the URL.
* @param host host of the URL
*/
@Public
@Evolving
public abstract void setHost(String host);
/**
* Get the port of the URL.
* @return port of the URL
*/
@Public
@Evolving
public abstract int getPort();
/**
* Set the port of the URL
* @param port port of the URL
*/
@Public
@Evolving
public abstract void setPort(int port);
/**
* Get the file of the URL.
* @return file of the URL
*/
@Public
@Evolving
public abstract String getFile();
public abstract void setScheme(String scheme);
public abstract void setHost(String host);
public abstract void setPort(int port);
/**
* Set the file of the URL.
* @param file file of the URL
*/
@Public
@Evolving
public abstract void setFile(String file);
}

View File

@ -18,10 +18,30 @@
package org.apache.hadoop.yarn.api.records;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
* <p><code>YarnClusterMetrics</code> represents cluster metrics.</p>
*
* <p>Currently only number of <code>NodeManager</code>s is provided.</p>
*/
@Public
@Stable
public interface YarnClusterMetrics {
/**
* Get the number of <code>NodeManager</code>s in the cluster.
* @return number of <code>NodeManager</code>s in the cluster
*/
@Public
@Stable
public abstract int getNumNodeManagers();
@Private
@Unstable
public abstract void setNumNodeManagers(int numNodeManagers);
}

View File

@ -19,10 +19,8 @@
package org.apache.hadoop.yarn.api.records.impl.pb;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProtoOrBuilder;
@ -32,8 +30,7 @@ public class ContainerIdPBImpl extends ContainerId {
ContainerIdProto.Builder builder = null;
boolean viaProto = false;
private ApplicationId applicationId = null;
private ApplicationAttemptId appAttemptId = null;
private ApplicationAttemptId applicationAttemptId = null;
public ContainerIdPBImpl() {
builder = ContainerIdProto.newBuilder();
@ -52,11 +49,10 @@ public synchronized ContainerIdProto getProto() {
}
private synchronized void mergeLocalToBuilder() {
if (this.applicationId != null && !((ApplicationIdPBImpl)applicationId).getProto().equals(builder.getAppId())) {
builder.setAppId(convertToProtoFormat(this.applicationId));
}
if (this.appAttemptId != null && !((ApplicationAttemptIdPBImpl)appAttemptId).getProto().equals(builder.getAppAttemptId())) {
builder.setAppAttemptId(convertToProtoFormat(this.appAttemptId));
if (this.applicationAttemptId != null && !
((ApplicationAttemptIdPBImpl)applicationAttemptId).getProto().equals(
builder.getAppAttemptId())) {
builder.setAppAttemptId(convertToProtoFormat(this.applicationAttemptId));
}
}
@ -87,61 +83,36 @@ public synchronized void setId(int id) {
maybeInitBuilder();
builder.setId((id));
}
@Override
public synchronized ApplicationId getAppId() {
ContainerIdProtoOrBuilder p = viaProto ? proto : builder;
if (this.applicationId != null) {
return this.applicationId;
}
if (!p.hasAppId()) {
return null;
}
this.applicationId = convertFromProtoFormat(p.getAppId());
return this.applicationId;
}
@Override
public synchronized ApplicationAttemptId getAppAttemptId() {
public synchronized ApplicationAttemptId getApplicationAttemptId() {
ContainerIdProtoOrBuilder p = viaProto ? proto : builder;
if (this.appAttemptId != null) {
return this.appAttemptId;
if (this.applicationAttemptId != null) {
return this.applicationAttemptId;
}
if (!p.hasAppAttemptId()) {
return null;
}
this.appAttemptId = convertFromProtoFormat(p.getAppAttemptId());
return this.appAttemptId;
this.applicationAttemptId = convertFromProtoFormat(p.getAppAttemptId());
return this.applicationAttemptId;
}
@Override
public synchronized void setAppId(ApplicationId appId) {
maybeInitBuilder();
if (appId == null)
builder.clearAppId();
this.applicationId = appId;
}
@Override
public synchronized void setAppAttemptId(ApplicationAttemptId atId) {
public synchronized void setApplicationAttemptId(ApplicationAttemptId atId) {
maybeInitBuilder();
if (atId == null)
builder.clearAppAttemptId();
this.appAttemptId = atId;
this.applicationAttemptId = atId;
}
private ApplicationAttemptIdPBImpl convertFromProtoFormat(ApplicationAttemptIdProto p) {
private ApplicationAttemptIdPBImpl convertFromProtoFormat(
ApplicationAttemptIdProto p) {
return new ApplicationAttemptIdPBImpl(p);
}
private ApplicationAttemptIdProto convertToProtoFormat(ApplicationAttemptId t) {
private ApplicationAttemptIdProto convertToProtoFormat(
ApplicationAttemptId t) {
return ((ApplicationAttemptIdPBImpl)t).getProto();
}
private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
return new ApplicationIdPBImpl(p);
}
private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
return ((ApplicationIdPBImpl)t).getProto();
}
}

View File

@ -41,8 +41,11 @@
public class ContainerLaunchContextPBImpl extends ProtoBase<ContainerLaunchContextProto> implements ContainerLaunchContext {
ContainerLaunchContextProto proto = ContainerLaunchContextProto.getDefaultInstance();
public class ContainerLaunchContextPBImpl
extends ProtoBase<ContainerLaunchContextProto>
implements ContainerLaunchContext {
ContainerLaunchContextProto proto =
ContainerLaunchContextProto.getDefaultInstance();
ContainerLaunchContextProto.Builder builder = null;
boolean viaProto = false;
@ -72,10 +75,14 @@ public ContainerLaunchContextProto getProto() {
}
private void mergeLocalToBuilder() {
if (this.containerId != null && !((ContainerIdPBImpl)containerId).getProto().equals(builder.getContainerId())) {
if (this.containerId != null &&
!((ContainerIdPBImpl)containerId).getProto().equals(
builder.getContainerId())) {
builder.setContainerId(convertToProtoFormat(this.containerId));
}
if (this.resource != null && !((ResourcePBImpl)this.resource).getProto().equals(builder.getResource())) {
if (this.resource != null &&
!((ResourcePBImpl)this.resource).getProto().equals(
builder.getResource())) {
builder.setResource(convertToProtoFormat(this.resource));
}
if (this.localResources != null) {
@ -131,22 +138,13 @@ public void setResource(Resource resource) {
builder.clearResource();
this.resource = resource;
}
@Override
public List<String> getCommandList() {
public List<String> getCommands() {
initCommands();
return this.commands;
}
@Override
public String getCommand(int index) {
initCommands();
return this.commands.get(index);
}
@Override
public int getCommandCount() {
initCommands();
return this.commands.size();
}
private void initCommands() {
if (this.commands != null) {
return;
@ -161,11 +159,12 @@ private void initCommands() {
}
@Override
public void addAllCommands(final List<String> command) {
if (command == null)
public void setCommands(final List<String> commands) {
if (commands == null)
return;
initCommands();
this.commands.addAll(command);
this.commands.clear();
this.commands.addAll(commands);
}
private void addCommandsToProto() {
@ -175,21 +174,7 @@ private void addCommandsToProto() {
return;
builder.addAllCommand(this.commands);
}
@Override
public void addCommand(String command) {
initCommands();
this.commands.add(command);
}
@Override
public void removeCommand(int index) {
initCommands();
this.commands.remove(index);
}
@Override
public void clearCommands() {
initCommands();
this.commands.clear();
}
@Override
public String getUser() {
ContainerLaunchContextProtoOrBuilder p = viaProto ? proto : builder;
@ -228,17 +213,13 @@ public void setContainerId(ContainerId containerId) {
builder.clearContainerId();
this.containerId = containerId;
}
@Override
public Map<String, LocalResource> getAllLocalResources() {
public Map<String, LocalResource> getLocalResources() {
initLocalResources();
return this.localResources;
}
@Override
public LocalResource getLocalResource(String key) {
initLocalResources();
return this.localResources.get(key);
}
private void initLocalResources() {
if (this.localResources != null) {
return;
@ -253,10 +234,12 @@ private void initLocalResources() {
}
@Override
public void addAllLocalResources(final Map<String, LocalResource> localResources) {
public void setLocalResources(
final Map<String, LocalResource> localResources) {
if (localResources == null)
return;
initLocalResources();
this.localResources.clear();
this.localResources.putAll(localResources);
}
@ -265,7 +248,8 @@ private void addLocalResourcesToProto() {
builder.clearLocalResources();
if (localResources == null)
return;
Iterable<StringLocalResourceMapProto> iterable = new Iterable<StringLocalResourceMapProto>() {
Iterable<StringLocalResourceMapProto> iterable =
new Iterable<StringLocalResourceMapProto>() {
@Override
public Iterator<StringLocalResourceMapProto> iterator() {
@ -281,7 +265,8 @@ public void remove() {
@Override
public StringLocalResourceMapProto next() {
String key = keyIter.next();
return StringLocalResourceMapProto.newBuilder().setKey(key).setValue(convertToProtoFormat(localResources.get(key))).build();
return StringLocalResourceMapProto.newBuilder().setKey(key).
setValue(convertToProtoFormat(localResources.get(key))).build();
}
@Override
@ -293,21 +278,7 @@ public boolean hasNext() {
};
builder.addAllLocalResources(iterable);
}
@Override
public void setLocalResource(String key, LocalResource val) {
initLocalResources();
this.localResources.put(key, val);
}
@Override
public void removeLocalResource(String key) {
initLocalResources();
this.localResources.remove(key);
}
@Override
public void clearLocalResources() {
initLocalResources();
this.localResources.clear();
}
@Override
public ByteBuffer getContainerTokens() {
ContainerLaunchContextProtoOrBuilder p = viaProto ? proto : builder;
@ -328,16 +299,12 @@ public void setContainerTokens(ByteBuffer containerTokens) {
builder.clearContainerTokens();
this.containerTokens = containerTokens;
}
@Override
public Map<String, ByteBuffer> getAllServiceData() {
public Map<String, ByteBuffer> getServiceData() {
initServiceData();
return this.serviceData;
}
@Override
public ByteBuffer getServiceData(String key) {
initServiceData();
return this.serviceData.get(key);
}
private void initServiceData() {
if (this.serviceData != null) {
@ -353,7 +320,7 @@ private void initServiceData() {
}
@Override
public void addAllServiceData(final Map<String, ByteBuffer> serviceData) {
public void setServiceData(final Map<String, ByteBuffer> serviceData) {
if (serviceData == null)
return;
initServiceData();
@ -365,7 +332,8 @@ private void addServiceDataToProto() {
builder.clearServiceData();
if (serviceData == null)
return;
Iterable<StringBytesMapProto> iterable = new Iterable<StringBytesMapProto>() {
Iterable<StringBytesMapProto> iterable =
new Iterable<StringBytesMapProto>() {
@Override
public Iterator<StringBytesMapProto> iterator() {
@ -381,7 +349,8 @@ public void remove() {
@Override
public StringBytesMapProto next() {
String key = keyIter.next();
return StringBytesMapProto.newBuilder().setKey(key).setValue(convertToProtoFormat(serviceData.get(key))).build();
return StringBytesMapProto.newBuilder().setKey(key).setValue(
convertToProtoFormat(serviceData.get(key))).build();
}
@Override
@ -393,31 +362,12 @@ public boolean hasNext() {
};
builder.addAllServiceData(iterable);
}
@Override
public void setServiceData(String key, ByteBuffer val) {
initServiceData();
this.serviceData.put(key, val);
}
@Override
public void removeServiceData(String key) {
initServiceData();
this.serviceData.remove(key);
}
@Override
public void clearServiceData() {
initServiceData();
this.serviceData.clear();
}
@Override
public Map<String, String> getAllEnv() {
public Map<String, String> getEnv() {
initEnv();
return this.env;
}
@Override
public String getEnv(String key) {
initEnv();
return this.env.get(key);
}
private void initEnv() {
if (this.env != null) {
@ -433,10 +383,11 @@ private void initEnv() {
}
@Override
public void addAllEnv(final Map<String, String> env) {
public void setEnv(final Map<String, String> env) {
if (env == null)
return;
initEnv();
this.env.clear();
this.env.putAll(env);
}
@ -445,7 +396,8 @@ private void addEnvToProto() {
builder.clearEnv();
if (env == null)
return;
Iterable<StringStringMapProto> iterable = new Iterable<StringStringMapProto>() {
Iterable<StringStringMapProto> iterable =
new Iterable<StringStringMapProto>() {
@Override
public Iterator<StringStringMapProto> iterator() {
@ -461,7 +413,8 @@ public void remove() {
@Override
public StringStringMapProto next() {
String key = keyIter.next();
return StringStringMapProto.newBuilder().setKey(key).setValue((env.get(key))).build();
return StringStringMapProto.newBuilder().setKey(key).setValue(
(env.get(key))).build();
}
@Override
@ -473,21 +426,6 @@ public boolean hasNext() {
};
builder.addAllEnv(iterable);
}
@Override
public void setEnv(String key, String val) {
initEnv();
this.env.put(key, val);
}
@Override
public void removeEnv(String key) {
initEnv();
this.env.remove(key);
}
@Override
public void clearEnv() {
initEnv();
this.env.clear();
}
private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
return new ResourcePBImpl(p);

View File

@ -31,7 +31,8 @@
public class ContainerStatusPBImpl extends ProtoBase<ContainerStatusProto> implements ContainerStatus {
public class ContainerStatusPBImpl extends ProtoBase<ContainerStatusProto>
implements ContainerStatus {
ContainerStatusProto proto = ContainerStatusProto.getDefaultInstance();
ContainerStatusProto.Builder builder = null;
boolean viaProto = false;
@ -116,13 +117,13 @@ public void setContainerId(ContainerId containerId) {
this.containerId = containerId;
}
@Override
public String getExitStatus() {
public int getExitStatus() {
ContainerStatusProtoOrBuilder p = viaProto ? proto : builder;
return (p.getExitStatus());
return p.getExitStatus();
}
@Override
public void setExitStatus(String exitStatus) {
public void setExitStatus(int exitStatus) {
maybeInitBuilder();
builder.setExitStatus(exitStatus);
}

View File

@ -250,7 +250,7 @@ message ContainerStatusProto {
optional ContainerIdProto container_id = 1;
optional ContainerStateProto state = 2;
optional string diagnostics = 3 [default = "N/A"];
optional string exit_status = 4 [default = "N/A"];
optional int32 exit_status = 4 [default = -1000];
}

View File

@ -351,6 +351,8 @@ public class YarnConfiguration extends Configuration {
public static final String NM_AUX_SERVICE_FMT =
NM_PREFIX + "aux-services.%s.class";
public static final int INVALID_CONTAINER_EXIT_STATUS = -1000;
public static final int ABORTED_CONTAINER_EXIT_STATUS = -100;
public YarnConfiguration() {
super();

View File

@ -68,28 +68,42 @@ public Resource getResource() {
@Override
public void write(DataOutput out) throws IOException {
LOG.debug("Writing ContainerTokenIdentifier to RPC layer");
out.writeInt(this.containerId.getAppId().getId());
out.writeInt(this.containerId.getAppAttemptId().getAttemptId());
ApplicationAttemptId applicationAttemptId =
containerId.getApplicationAttemptId();
ApplicationId applicationId = applicationAttemptId.getApplicationId();
out.writeLong(applicationId.getClusterTimestamp());
out.writeInt(applicationId.getId());
out.writeInt(applicationAttemptId.getAttemptId());
out.writeInt(this.containerId.getId());
// TODO: Cluster time-stamp?
out.writeUTF(this.nmHostName);
out.writeInt(this.resource.getMemory()); // TODO: more resources.
out.writeInt(this.resource.getMemory());
}
@Override
public void readFields(DataInput in) throws IOException {
this.containerId = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ContainerId.class);
this.containerId.setAppId(RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class));
this.containerId.setAppAttemptId(RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationAttemptId.class));
this.containerId.getAppId().setId(in.readInt());
this.containerId.getAppAttemptId().setApplicationId(this.containerId.getAppId());
this.containerId.getAppAttemptId().setAttemptId(in.readInt());
this.containerId =
RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
ContainerId.class);
ApplicationAttemptId applicationAttemptId =
RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
ApplicationAttemptId.class);
ApplicationId applicationId =
RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
ApplicationId.class);
applicationId.setClusterTimestamp(in.readLong());
applicationId.setId(in.readInt());
applicationAttemptId.setApplicationId(applicationId);
applicationAttemptId.setAttemptId(in.readInt());
this.containerId.setApplicationAttemptId(applicationAttemptId);
this.containerId.setId(in.readInt());
this.nmHostName = in.readUTF();
this.resource = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(Resource.class);
this.resource.setMemory(in.readInt()); // TODO: more resources.
this.resource =
RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
Resource.class);
this.resource.setMemory(in.readInt());
}
@SuppressWarnings("static-access")
@Override
public Text getKind() {
return this.KIND;

View File

@ -152,9 +152,8 @@ public static ApplicationId convert(long clustertimestamp, CharSequence id) {
public static ContainerId newContainerId(ApplicationAttemptId appAttemptId,
int containerId) {
ContainerId id = recordFactory.newRecordInstance(ContainerId.class);
id.setAppId(appAttemptId.getApplicationId());
id.setId(containerId);
id.setAppAttemptId(appAttemptId);
id.setApplicationAttemptId(appAttemptId);
return id;
}
@ -171,9 +170,8 @@ public static ContainerId newContainerId(RecordFactory recordFactory,
ApplicationId appId, ApplicationAttemptId appAttemptId,
int containerId) {
ContainerId id = recordFactory.newRecordInstance(ContainerId.class);
id.setAppId(appId);
id.setId(containerId);
id.setAppAttemptId(appAttemptId);
id.setApplicationAttemptId(appAttemptId);
return id;
}
@ -181,8 +179,7 @@ public static ContainerId newContainerId(RecordFactory recordFactory,
ApplicationAttemptId appAttemptId,
int containerId) {
ContainerId id = recordFactory.newRecordInstance(ContainerId.class);
id.setAppAttemptId(appAttemptId);
id.setAppId(appAttemptId.getApplicationId());
id.setApplicationAttemptId(appAttemptId);
id.setId(containerId);
return id;
}

View File

@ -29,6 +29,7 @@
import java.util.Map.Entry;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.URL;
@ -130,6 +131,20 @@ private static ApplicationId toApplicationId(RecordFactory recordFactory,
return appId;
}
private static ApplicationAttemptId toApplicationAttemptId(
RecordFactory recordFactory,
Iterator<String> it) {
ApplicationId appId =
recordFactory.newRecordInstance(ApplicationId.class);
appId.setClusterTimestamp(Long.parseLong(it.next()));
appId.setId(Integer.parseInt(it.next()));
ApplicationAttemptId appAttemptId =
recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId.setApplicationId(appId);
appAttemptId.setAttemptId(Integer.parseInt(it.next()));
return appAttemptId;
}
public static String toString(ContainerId cId) {
return cId.toString();
}
@ -138,10 +153,11 @@ public static ContainerId toContainerId(RecordFactory recordFactory,
String containerIdStr) {
Iterator<String> it = _split(containerIdStr).iterator();
it.next(); // prefix. TODO: Validate container prefix
ApplicationId appID = toApplicationId(recordFactory, it);
ApplicationAttemptId appAttemptID =
toApplicationAttemptId(recordFactory, it);
ContainerId containerId =
recordFactory.newRecordInstance(ContainerId.class);
containerId.setAppId(appID);
containerId.setApplicationAttemptId(appAttemptID);
containerId.setId(Integer.parseInt(it.next()));
return containerId;
}

View File

@ -32,6 +32,7 @@
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
@ -81,21 +82,34 @@ private void test(String rpcClass) throws Exception {
ContainerManager proxy = (ContainerManager)
rpc.getProxy(ContainerManager.class,
NetUtils.createSocketAddr("localhost:" + server.getPort()), conf);
ContainerLaunchContext containerLaunchContext = recordFactory.newRecordInstance(ContainerLaunchContext.class);
ContainerLaunchContext containerLaunchContext =
recordFactory.newRecordInstance(ContainerLaunchContext.class);
containerLaunchContext.setUser("dummy-user");
containerLaunchContext.setContainerId(recordFactory.newRecordInstance(ContainerId.class));
containerLaunchContext.getContainerId().setAppId(recordFactory.newRecordInstance(ApplicationId.class));
containerLaunchContext.getContainerId().getAppId().setId(0);
containerLaunchContext.getContainerId().setId(100);
containerLaunchContext.setResource(recordFactory.newRecordInstance(Resource.class));
ContainerId containerId =
recordFactory.newRecordInstance(ContainerId.class);
ApplicationId applicationId =
recordFactory.newRecordInstance(ApplicationId.class);
ApplicationAttemptId applicationAttemptId =
recordFactory.newRecordInstance(ApplicationAttemptId.class);
applicationId.setClusterTimestamp(0);
applicationId.setId(0);
applicationAttemptId.setApplicationId(applicationId);
applicationAttemptId.setAttemptId(0);
containerId.setApplicationAttemptId(applicationAttemptId);
containerId.setId(100);
containerLaunchContext.setContainerId(containerId);
containerLaunchContext.setResource(
recordFactory.newRecordInstance(Resource.class));
// containerLaunchContext.env = new HashMap<CharSequence, CharSequence>();
// containerLaunchContext.command = new ArrayList<CharSequence>();
StartContainerRequest scRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
StartContainerRequest scRequest =
recordFactory.newRecordInstance(StartContainerRequest.class);
scRequest.setContainerLaunchContext(containerLaunchContext);
proxy.startContainer(scRequest);
GetContainerStatusRequest gcsRequest = recordFactory.newRecordInstance(GetContainerStatusRequest.class);
GetContainerStatusRequest gcsRequest =
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
gcsRequest.setContainerId(containerLaunchContext.getContainerId());
GetContainerStatusResponse response = proxy.getContainerStatus(gcsRequest);
ContainerStatus status = response.getStatus();
@ -118,7 +132,7 @@ private void test(String rpcClass) throws Exception {
server.close();
Assert.assertNotNull(status);
Assert.assertEquals(ContainerState.RUNNING, status.getState().RUNNING);
Assert.assertEquals(ContainerState.RUNNING, status.getState());
}
public class DummyContainerManager implements ContainerManager {
@ -126,28 +140,35 @@ public class DummyContainerManager implements ContainerManager {
private ContainerStatus status = null;
@Override
public GetContainerStatusResponse getContainerStatus(GetContainerStatusRequest request) throws YarnRemoteException {
GetContainerStatusResponse response = recordFactory.newRecordInstance(GetContainerStatusResponse.class);
public GetContainerStatusResponse getContainerStatus(
GetContainerStatusRequest request)
throws YarnRemoteException {
GetContainerStatusResponse response =
recordFactory.newRecordInstance(GetContainerStatusResponse.class);
response.setStatus(status);
return response;
}
@Override
public StartContainerResponse startContainer(StartContainerRequest request) throws YarnRemoteException {
public StartContainerResponse startContainer(StartContainerRequest request)
throws YarnRemoteException {
ContainerLaunchContext container = request.getContainerLaunchContext();
StartContainerResponse response = recordFactory.newRecordInstance(StartContainerResponse.class);
StartContainerResponse response =
recordFactory.newRecordInstance(StartContainerResponse.class);
status = recordFactory.newRecordInstance(ContainerStatus.class);
status.setState(ContainerState.RUNNING);
status.setContainerId(container.getContainerId());
status.setExitStatus(String.valueOf(0));
status.setExitStatus(0);
return response;
}
@Override
public StopContainerResponse stopContainer(StopContainerRequest request) throws YarnRemoteException {
public StopContainerResponse stopContainer(StopContainerRequest request)
throws YarnRemoteException {
Exception e = new Exception(EXCEPTION_MSG,
new Exception(EXCEPTION_CAUSE));
throw YarnRemoteExceptionFactoryProvider.getYarnRemoteExceptionFactory(null).createYarnRemoteException(e);
throw YarnRemoteExceptionFactoryProvider
.getYarnRemoteExceptionFactory(null).createYarnRemoteException(e);
}
}
}

View File

@ -40,8 +40,7 @@ private ContainerId createContainerId(long clusterTimestamp, int appIdInt,
ApplicationAttemptId appAttemptId =
createAppAttemptId(appId, appAttemptIdInt);
ContainerId containerId = Records.newRecord(ContainerId.class);
containerId.setAppAttemptId(appAttemptId);
containerId.setAppId(appId);
containerId.setApplicationAttemptId(appAttemptId);
containerId.setId(containerIdInt);
return containerId;
}

View File

@ -97,7 +97,9 @@ public int launchContainer(Container container,
// create container dirs on all disks
String containerIdStr = ConverterUtils.toString(containerId);
String appIdStr =
ConverterUtils.toString(container.getContainerID().getAppId());
ConverterUtils.toString(
container.getContainerID().getApplicationAttemptId().
getApplicationId());
String[] sLocalDirs =
getConf().getStrings(YarnConfiguration.NM_LOCAL_DIRS, YarnConfiguration.DEFAULT_NM_LOCAL_DIRS);
for (String sLocalDir : sLocalDirs) {

View File

@ -250,7 +250,8 @@ public StartContainerResponse startContainer(StartContainerRequest request)
Container container =
new ContainerImpl(this.dispatcher, launchContext, credentials, metrics);
ContainerId containerID = launchContext.getContainerId();
ApplicationId applicationID = containerID.getAppId();
ApplicationId applicationID =
containerID.getApplicationAttemptId().getApplicationId();
if (context.getContainers().putIfAbsent(containerID, container) != null) {
NMAuditLogger.logFailure(launchContext.getUser(),
AuditConstants.START_CONTAINER, "ContainerManagerImpl",
@ -305,7 +306,8 @@ public StopContainerResponse stopContainer(StopContainerRequest request)
NMAuditLogger.logFailure(userName,
AuditConstants.STOP_CONTAINER, "ContainerManagerImpl",
"Trying to stop unknown container!",
containerID.getAppId(), containerID);
containerID.getApplicationAttemptId().getApplicationId(),
containerID);
return response; // Return immediately.
}
dispatcher.getEventHandler().handle(
@ -317,7 +319,8 @@ public StopContainerResponse stopContainer(StopContainerRequest request)
// should be the same or should be rejected by auth before here.
NMAuditLogger.logSuccess(container.getUser(),
AuditConstants.STOP_CONTAINER, "ContainerManageImpl",
containerID.getAppId(), containerID);
containerID.getApplicationAttemptId().getApplicationId(),
containerID);
// TODO: Move this code to appropriate place once kill_container is
// implemented.

View File

@ -25,7 +25,7 @@ public class ApplicationContainerFinishedEvent extends ApplicationEvent {
public ApplicationContainerFinishedEvent(
ContainerId containerID) {
super(containerID.getAppId(),
super(containerID.getApplicationAttemptId().getApplicationId(),
ApplicationEventType.APPLICATION_CONTAINER_FINISHED);
this.containerID = containerID;
}

View File

@ -25,7 +25,7 @@ public class ApplicationInitEvent extends ApplicationEvent {
private final Container container;
public ApplicationInitEvent(Container container) {
super(container.getContainerID().getAppId(),
super(container.getContainerID().getApplicationAttemptId().getApplicationId(),
ApplicationEventType.INIT_APPLICATION);
this.container = container;
}

View File

@ -40,6 +40,7 @@
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.factories.RecordFactory;
@ -53,9 +54,7 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEventType;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourceRequest;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationCleanupEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationRequestEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.event.LogAggregatorContainerFinishedEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainerStartMonitoringEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainerStopMonitoringEvent;
@ -75,7 +74,7 @@ public class ContainerImpl implements Container {
private final Credentials credentials;
private final NodeManagerMetrics metrics;
private final ContainerLaunchContext launchContext;
private String exitCode = "NA";
private int exitCode = YarnConfiguration.INVALID_CONTAINER_EXIT_STATUS;
private final StringBuilder diagnostics;
private static final Log LOG = LogFactory.getLog(Container.class);
@ -345,7 +344,7 @@ public ContainerStatus cloneAndGetContainerStatus() {
containerStatus.setState(getCurrentState());
containerStatus.setContainerId(this.launchContext.getContainerId());
containerStatus.setDiagnostics(diagnostics.toString());
containerStatus.setExitStatus(String.valueOf(exitCode));
containerStatus.setExitStatus(exitCode);
return containerStatus;
} finally {
this.readLock.unlock();
@ -360,7 +359,8 @@ private void finished() {
metrics.completedContainer();
NMAuditLogger.logSuccess(getUser(),
AuditConstants.FINISH_SUCCESS_CONTAINER, "ContainerImpl",
getContainerID().getAppId(), getContainerID());
getContainerID().getApplicationAttemptId().getApplicationId(),
getContainerID());
break;
case EXITED_WITH_FAILURE:
metrics.endRunningContainer();
@ -370,7 +370,8 @@ private void finished() {
NMAuditLogger.logFailure(getUser(),
AuditConstants.FINISH_FAILED_CONTAINER, "ContainerImpl",
"Container failed with state: " + getContainerState(),
getContainerID().getAppId(), getContainerID());
getContainerID().getApplicationAttemptId().getApplicationId(),
getContainerID());
break;
case CONTAINER_CLEANEDUP_AFTER_KILL:
metrics.endRunningContainer();
@ -379,13 +380,15 @@ private void finished() {
metrics.killedContainer();
NMAuditLogger.logSuccess(getUser(),
AuditConstants.FINISH_KILLED_CONTAINER, "ContainerImpl",
getContainerID().getAppId(), getContainerID());
getContainerID().getApplicationAttemptId().getApplicationId(),
getContainerID());
}
metrics.releaseContainer(getLaunchContext().getResource());
// Inform the application
ContainerId containerID = getContainerID();
@SuppressWarnings("rawtypes")
EventHandler eventHandler = dispatcher.getEventHandler();
eventHandler.handle(new ApplicationContainerFinishedEvent(containerID));
// Remove the container from the resource-monitor
@ -433,20 +436,21 @@ public ContainerState transition(ContainerImpl container,
container.metrics.initingContainer();
// Inform the AuxServices about the opaque serviceData
Map<String,ByteBuffer> csd = ctxt.getAllServiceData();
Map<String,ByteBuffer> csd = ctxt.getServiceData();
if (csd != null) {
// This can happen more than once per Application as each container may
// have distinct service data
for (Map.Entry<String,ByteBuffer> service : csd.entrySet()) {
container.dispatcher.getEventHandler().handle(
new AuxServicesEvent(AuxServicesEventType.APPLICATION_INIT,
ctxt.getUser(), ctxt.getContainerId().getAppId(),
ctxt.getUser(),
ctxt.getContainerId().getApplicationAttemptId().getApplicationId(),
service.getKey().toString(), service.getValue()));
}
}
// Send requests for public, private resources
Map<String,LocalResource> cntrRsrc = ctxt.getAllLocalResources();
Map<String,LocalResource> cntrRsrc = ctxt.getLocalResources();
if (!cntrRsrc.isEmpty()) {
try {
for (Map.Entry<String,LocalResource> rsrc : cntrRsrc.entrySet()) {
@ -562,7 +566,7 @@ static class ExitedWithFailureTransition extends ContainerTransition {
@Override
public void transition(ContainerImpl container, ContainerEvent event) {
ContainerExitEvent exitEvent = (ContainerExitEvent) event;
container.exitCode = String.valueOf(exitEvent.getExitCode());
container.exitCode = exitEvent.getExitCode();
// TODO: Add containerWorkDir to the deletion service.
// TODO: Add containerOuputDir to the deletion service.
@ -640,7 +644,7 @@ static class ContainerKilledTransition implements
@Override
public void transition(ContainerImpl container, ContainerEvent event) {
ContainerExitEvent exitEvent = (ContainerExitEvent) event;
container.exitCode = String.valueOf(exitEvent.getExitCode());
container.exitCode = exitEvent.getExitCode();
// The process/process-grp is killed. Decrement reference counts and
// cleanup resources

View File

@ -89,8 +89,8 @@ public Integer call() {
final Map<Path,String> localResources = container.getLocalizedResources();
String containerIdStr = ConverterUtils.toString(container.getContainerID());
final String user = launchContext.getUser();
final Map<String,String> env = launchContext.getAllEnv();
final List<String> command = launchContext.getCommandList();
final Map<String,String> env = launchContext.getEnv();
final List<String> command = launchContext.getCommands();
int ret = -1;
try {
@ -107,10 +107,9 @@ public Integer call() {
newCmds.add(str.replace(ApplicationConstants.LOG_DIR_EXPANSION_VAR,
containerLogDir.toUri().getPath()));
}
launchContext.clearCommands();
launchContext.addAllCommands(newCmds);
launchContext.setCommands(newCmds);
Map<String, String> envs = launchContext.getAllEnv();
Map<String, String> envs = launchContext.getEnv();
Map<String, String> newEnvs = new HashMap<String, String>(envs.size());
for (Entry<String, String> entry : envs.entrySet()) {
newEnvs.put(
@ -119,8 +118,7 @@ public Integer call() {
ApplicationConstants.LOG_DIR_EXPANSION_VAR,
containerLogDir.toUri().getPath()));
}
launchContext.clearEnv();
launchContext.addAllEnv(newEnvs);
launchContext.setEnv(newEnvs);
// /////////////////////////// End of variable expansion
FileContext lfs = FileContext.getLocalFSFileContext();
@ -170,7 +168,7 @@ public Integer call() {
containerWorkDir, FINAL_CONTAINER_TOKENS_FILE).toUri().getPath());
writeLaunchEnv(containerScriptOutStream, env, localResources,
launchContext.getCommandList(), appDirs);
launchContext.getCommands(), appDirs);
// /////////// End of writing out container-script
// /////////// Write out the container-tokens in the nmPrivate space.

View File

@ -103,7 +103,8 @@ public void handle(ContainersLauncherEvent event) {
switch (event.getType()) {
case LAUNCH_CONTAINER:
Application app =
context.getApplications().get(containerId.getAppId());
context.getApplications().get(
containerId.getApplicationAttemptId().getApplicationId());
ContainerLaunch launch =
new ContainerLaunch(getConfig(), dispatcher, exec, app,
event.getContainer());

View File

@ -292,7 +292,7 @@ public void handle(LocalizationEvent event) {
for (Map.Entry<LocalResourceVisibility, Collection<LocalResourceRequest>> e :
rsrcs.entrySet()) {
tracker = getLocalResourcesTracker(e.getKey(), c.getUser(),
c.getContainerID().getAppId());
c.getContainerID().getApplicationAttemptId().getApplicationId());
for (LocalResourceRequest req : e.getValue()) {
tracker.handle(new ResourceRequestEvent(req, e.getKey(), ctxt));
}
@ -316,7 +316,7 @@ public void handle(LocalizationEvent event) {
for (Map.Entry<LocalResourceVisibility, Collection<LocalResourceRequest>> e :
rsrcs.entrySet()) {
tracker = getLocalResourcesTracker(e.getKey(), c.getUser(),
c.getContainerID().getAppId());
c.getContainerID().getApplicationAttemptId().getApplicationId());
for (LocalResourceRequest req : e.getValue()) {
tracker.handle(new ResourceReleaseEvent(req, c.getContainerID()));
}
@ -326,7 +326,8 @@ public void handle(LocalizationEvent event) {
userName = c.getUser();
String containerIDStr = c.toString();
appIDStr =
ConverterUtils.toString(c.getContainerID().getAppId());
ConverterUtils.toString(
c.getContainerID().getApplicationAttemptId().getApplicationId());
for (Path localDir : localDirs) {
// Delete the user-owned container-dir
@ -789,7 +790,9 @@ public void run() {
// 2) exec initApplication and wait
exec.startLocalizer(nmPrivateCTokensPath, localizationServerAddress,
context.getUser(),
ConverterUtils.toString(context.getContainerId().getAppId()),
ConverterUtils.toString(
context.getContainerId().
getApplicationAttemptId().getApplicationId()),
localizerId, localDirs);
// TODO handle ExitCodeException separately?
} catch (Exception e) {

View File

@ -90,8 +90,11 @@ public LogValue(String[] rootLogDirs, ContainerId containerId) {
public void write(DataOutputStream out) throws IOException {
for (String rootLogDir : this.rootLogDirs) {
File appLogDir =
new File(rootLogDir, ConverterUtils.toString(this.containerId
.getAppId()));
new File(rootLogDir,
ConverterUtils.toString(
this.containerId.getApplicationAttemptId().
getApplicationId())
);
File containerLogDir =
new File(appLogDir, ConverterUtils.toString(this.containerId));

View File

@ -172,11 +172,13 @@ private void stopContainer(ContainerId containerId, String exitCode) {
// A container is complete. Put this containers' logs up for aggregation if
// this containers' logs are needed.
if (!this.appLogAggregators.containsKey(containerId.getAppId())) {
if (!this.appLogAggregators.containsKey(
containerId.getApplicationAttemptId().getApplicationId())) {
throw new YarnException("Application is not initialized yet for "
+ containerId);
}
this.appLogAggregators.get(containerId.getAppId())
this.appLogAggregators.get(
containerId.getApplicationAttemptId().getApplicationId())
.startContainerLogAggregation(containerId, exitCode.equals("0"));
}

View File

@ -23,10 +23,10 @@
public class LogAggregatorContainerFinishedEvent extends LogAggregatorEvent {
private final ContainerId containerId;
private final String exitCode;
private final int exitCode;
public LogAggregatorContainerFinishedEvent(ContainerId containerId,
String exitCode) {
int exitCode) {
super(LogAggregatorEventType.CONTAINER_FINISHED);
this.containerId = containerId;
this.exitCode = exitCode;
@ -36,7 +36,7 @@ public ContainerId getContainerId() {
return this.containerId;
}
public String getExitCode() {
public int getExitCode() {
return this.exitCode;
}

View File

@ -95,7 +95,8 @@ protected void render(Block html) {
logFile =
new File(this.logsSelector
.getLocalPathToRead(
ConverterUtils.toString(containerId.getAppId())
ConverterUtils.toString(
containerId.getApplicationAttemptId().getApplicationId())
+ Path.SEPARATOR + $(CONTAINER_ID)
+ Path.SEPARATOR
+ $(CONTAINER_LOG_TYPE), this.conf).toUri()
@ -176,7 +177,9 @@ protected void render(Block html) {
conf.getStrings(YarnConfiguration.NM_LOG_DIRS, YarnConfiguration.DEFAULT_NM_LOG_DIRS);
List<File> containerLogDirs = new ArrayList<File>(logDirs.length);
for (String logDir : logDirs) {
String appIdStr = ConverterUtils.toString(containerId.getAppId());
String appIdStr =
ConverterUtils.toString(
containerId.getApplicationAttemptId().getApplicationId());
File appLogDir = new File(logDir, appIdStr);
String containerIdStr = ConverterUtils.toString(containerId);
containerLogDirs.add(new File(appLogDir, containerIdStr));

View File

@ -25,6 +25,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.nodemanager.Context;
@ -69,10 +70,14 @@ protected void render(Block html) {
ConverterUtils.toContainerId(this.recordFactory, $(CONTAINER_ID));
Container container = this.nmContext.getContainers().get(containerID);
ContainerStatus containerData = container.cloneAndGetContainerStatus();
int exitCode = containerData.getExitStatus();
String exiStatus =
(exitCode == YarnConfiguration.INVALID_CONTAINER_EXIT_STATUS) ?
"N/A" : String.valueOf(exitCode);
info("Container information")
._("ContainerID", $(CONTAINER_ID))
._("ContainerState", container.getContainerState())
._("ExitStatus", containerData.getExitStatus())
._("ExitStatus", exiStatus)
._("Diagnostics", containerData.getDiagnostics())
._("User", container.getUser())
._("TotalMemoryNeeded",

View File

@ -21,8 +21,6 @@
import java.io.File;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.NodeHealthCheckerService;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path;
@ -37,7 +35,6 @@
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
@ -49,8 +46,8 @@
public class TestEventFlow {
private static final Log LOG = LogFactory.getLog(TestEventFlow.class);
private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
private static final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
private static File localDir = new File("target",
TestEventFlow.class.getName() + "-localDir").getAbsoluteFile();
@ -77,7 +74,8 @@ public void testSuccessfulContainerLaunch() throws InterruptedException,
YarnConfiguration conf = new YarnConfiguration();
conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
remoteLogDir.getAbsolutePath());
ContainerExecutor exec = new DefaultContainerExecutor();
exec.setConf(conf);
@ -100,27 +98,36 @@ protected void startStatusUpdater() {
};
DummyContainerManager containerManager =
new DummyContainerManager(context, exec, del, nodeStatusUpdater, metrics, containerTokenSecretManager);
new DummyContainerManager(context, exec, del, nodeStatusUpdater,
metrics, containerTokenSecretManager);
containerManager.init(conf);
containerManager.start();
ContainerLaunchContext launchContext = recordFactory.newRecordInstance(ContainerLaunchContext.class);
ContainerLaunchContext launchContext =
recordFactory.newRecordInstance(ContainerLaunchContext.class);
ContainerId cID = recordFactory.newRecordInstance(ContainerId.class);
cID.setAppId(recordFactory.newRecordInstance(ApplicationId.class));
ApplicationAttemptId atId = recordFactory.newRecordInstance(ApplicationAttemptId.class);
atId.setApplicationId(cID.getAppId());
cID.setAppAttemptId(atId);
ApplicationId applicationId =
recordFactory.newRecordInstance(ApplicationId.class);
applicationId.setClusterTimestamp(0);
applicationId.setId(0);
ApplicationAttemptId applicationAttemptId =
recordFactory.newRecordInstance(ApplicationAttemptId.class);
applicationAttemptId.setApplicationId(applicationId);
applicationAttemptId.setAttemptId(0);
cID.setApplicationAttemptId(applicationAttemptId);
launchContext.setContainerId(cID);
launchContext.setUser("testing");
launchContext.setResource(recordFactory.newRecordInstance(Resource.class));
StartContainerRequest request = recordFactory.newRecordInstance(StartContainerRequest.class);
StartContainerRequest request =
recordFactory.newRecordInstance(StartContainerRequest.class);
request.setContainerLaunchContext(launchContext);
containerManager.startContainer(request);
BaseContainerManagerTest.waitForContainerState(containerManager, cID,
ContainerState.RUNNING);
StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
StopContainerRequest stopRequest =
recordFactory.newRecordInstance(StopContainerRequest.class);
stopRequest.setContainerId(cID);
containerManager.stopContainer(stopRequest);
BaseContainerManagerTest.waitForContainerState(containerManager, cID,

View File

@ -134,7 +134,8 @@ private Map<ApplicationId, List<ContainerStatus>> getAppToContainerStatusMap(
Map<ApplicationId, List<ContainerStatus>> map =
new HashMap<ApplicationId, List<ContainerStatus>>();
for (ContainerStatus cs : containers) {
ApplicationId applicationId = cs.getContainerId().getAppId();
ApplicationId applicationId =
cs.getContainerId().getApplicationAttemptId().getApplicationId();
List<ContainerStatus> appContainers = map.get(applicationId);
if (appContainers == null) {
appContainers = new ArrayList<ContainerStatus>();
@ -159,8 +160,7 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
// Give a container to the NM.
applicationID.setId(heartBeatID);
appAttemptID.setApplicationId(applicationID);
firstContainerID.setAppId(applicationID);
firstContainerID.setAppAttemptId(appAttemptID);
firstContainerID.setApplicationAttemptId(appAttemptID);
firstContainerID.setId(heartBeatID);
ContainerLaunchContext launchContext = recordFactory
.newRecordInstance(ContainerLaunchContext.class);
@ -184,8 +184,7 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
// Give another container to the NM.
applicationID.setId(heartBeatID);
appAttemptID.setApplicationId(applicationID);
secondContainerID.setAppId(applicationID);
secondContainerID.setAppAttemptId(appAttemptID);
secondContainerID.setApplicationAttemptId(appAttemptID);
secondContainerID.setId(heartBeatID);
ContainerLaunchContext launchContext = recordFactory
.newRecordInstance(ContainerLaunchContext.class);

View File

@ -23,7 +23,11 @@
import java.io.FileReader;
import java.io.IOException;
import java.io.PrintWriter;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import junit.framework.Assert;
@ -67,6 +71,20 @@ public TestContainerManager() throws UnsupportedFileSystemException {
LOG = LogFactory.getLog(TestContainerManager.class);
}
private ContainerId createContainerId() {
ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
appId.setClusterTimestamp(0);
appId.setId(0);
ApplicationAttemptId appAttemptId =
recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId.setApplicationId(appId);
appAttemptId.setAttemptId(1);
ContainerId containerId =
recordFactory.newRecordInstance(ContainerId.class);
containerId.setApplicationAttemptId(appAttemptId);
return containerId;
}
@Test
public void testContainerManagerInitialization() throws IOException {
@ -75,14 +93,9 @@ public void testContainerManagerInitialization() throws IOException {
// Just do a query for a non-existing container.
boolean throwsException = false;
try {
GetContainerStatusRequest request = recordFactory.newRecordInstance(GetContainerStatusRequest.class);
ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId.setApplicationId(appId);
appAttemptId.setAttemptId(1);
ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
cId.setAppId(appId);
cId.setAppAttemptId(appAttemptId);
GetContainerStatusRequest request =
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
ContainerId cId = createContainerId();
request.setContainerId(cId);
containerManager.getContainerStatus(request);
} catch (YarnRemoteException e) {
@ -107,20 +120,14 @@ public void testContainerSetup() throws IOException, InterruptedException {
ContainerLaunchContext container = recordFactory.newRecordInstance(ContainerLaunchContext.class);
// ////// Construct the Container-id
ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId.setApplicationId(appId);
appAttemptId.setAttemptId(1);
ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
cId.setAppId(appId);
cId.setAppAttemptId(appAttemptId);
ContainerId cId = createContainerId();
container.setContainerId(cId);
container.setUser(user);
// ////// Construct the container-spec.
ContainerLaunchContext containerLaunchContext = recordFactory.newRecordInstance(ContainerLaunchContext.class);
// containerLaunchContext.resources = new HashMap<CharSequence, LocalResource>();
ContainerLaunchContext containerLaunchContext =
recordFactory.newRecordInstance(ContainerLaunchContext.class);
URL resource_alpha =
ConverterUtils.getYarnUrlFromPath(localFS
.makeQualified(new Path(file.getAbsolutePath())));
@ -131,14 +138,17 @@ public void testContainerSetup() throws IOException, InterruptedException {
rsrc_alpha.setType(LocalResourceType.FILE);
rsrc_alpha.setTimestamp(file.lastModified());
String destinationFile = "dest_file";
containerLaunchContext.setLocalResource(destinationFile, rsrc_alpha);
Map<String, LocalResource> localResources =
new HashMap<String, LocalResource>();
localResources.put(destinationFile, rsrc_alpha);
containerLaunchContext.setLocalResources(localResources);
containerLaunchContext.setUser(container.getUser());
containerLaunchContext.setContainerId(container.getContainerId());
containerLaunchContext.setResource(recordFactory
.newRecordInstance(Resource.class));
// containerLaunchContext.command = new ArrayList<CharSequence>();
StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
StartContainerRequest startRequest =
recordFactory.newRecordInstance(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext);
containerManager.startContainer(startRequest);
@ -147,7 +157,7 @@ public void testContainerSetup() throws IOException, InterruptedException {
ContainerState.COMPLETE);
// Now ascertain that the resources are localised correctly.
// TODO: Don't we need clusterStamp in localDir?
ApplicationId appId = cId.getApplicationAttemptId().getApplicationId();
String appIDStr = ConverterUtils.toString(appId);
String containerIDStr = ConverterUtils.toString(cId);
File userCacheDir = new File(localDir, ContainerLocalizer.USERCACHE);
@ -187,41 +197,41 @@ public void testContainerLaunchAndStop() throws IOException,
PrintWriter fileWriter = new PrintWriter(scriptFile);
File processStartFile =
new File(tmpDir, "start_file.txt").getAbsoluteFile();
fileWriter.write("\numask 0"); // So that start file is readable by the test.
fileWriter.write("\numask 0"); // So that start file is readable by the test
fileWriter.write("\necho Hello World! > " + processStartFile);
fileWriter.write("\necho $$ >> " + processStartFile);
fileWriter.write("\nexec sleep 100");
fileWriter.close();
ContainerLaunchContext containerLaunchContext = recordFactory.newRecordInstance(ContainerLaunchContext.class);
ContainerLaunchContext containerLaunchContext =
recordFactory.newRecordInstance(ContainerLaunchContext.class);
// ////// Construct the Container-id
ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId.setApplicationId(appId);
appAttemptId.setAttemptId(1);
ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
cId.setAppId(appId);
cId.setAppAttemptId(appAttemptId);
ContainerId cId = createContainerId();
containerLaunchContext.setContainerId(cId);
containerLaunchContext.setUser(user);
// containerLaunchContext.resources =new HashMap<CharSequence, LocalResource>();
URL resource_alpha =
ConverterUtils.getYarnUrlFromPath(localFS
.makeQualified(new Path(scriptFile.getAbsolutePath())));
LocalResource rsrc_alpha = recordFactory.newRecordInstance(LocalResource.class);
LocalResource rsrc_alpha =
recordFactory.newRecordInstance(LocalResource.class);
rsrc_alpha.setResource(resource_alpha);
rsrc_alpha.setSize(-1);
rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
rsrc_alpha.setType(LocalResourceType.FILE);
rsrc_alpha.setTimestamp(scriptFile.lastModified());
String destinationFile = "dest_file";
containerLaunchContext.setLocalResource(destinationFile, rsrc_alpha);
Map<String, LocalResource> localResources =
new HashMap<String, LocalResource>();
localResources.put(destinationFile, rsrc_alpha);
containerLaunchContext.setLocalResources(localResources);
containerLaunchContext.setUser(containerLaunchContext.getUser());
containerLaunchContext.addCommand("/bin/bash");
containerLaunchContext.addCommand(scriptFile.getAbsolutePath());
List<String> commands = new ArrayList<String>();
commands.add("/bin/bash");
commands.add(scriptFile.getAbsolutePath());
containerLaunchContext.setCommands(commands);
containerLaunchContext.setResource(recordFactory
.newRecordInstance(Resource.class));
containerLaunchContext.getResource().setMemory(100 * 1024 * 1024);
@ -264,10 +274,12 @@ public void testContainerLaunchAndStop() throws IOException,
BaseContainerManagerTest.waitForContainerState(containerManager, cId,
ContainerState.COMPLETE);
GetContainerStatusRequest gcsRequest = recordFactory.newRecordInstance(GetContainerStatusRequest.class);
GetContainerStatusRequest gcsRequest =
recordFactory.newRecordInstance(GetContainerStatusRequest.class);
gcsRequest.setContainerId(cId);
ContainerStatus containerStatus = containerManager.getContainerStatus(gcsRequest).getStatus();
Assert.assertEquals(String.valueOf(ExitCode.KILLED.getExitCode()),
ContainerStatus containerStatus =
containerManager.getContainerStatus(gcsRequest).getStatus();
Assert.assertEquals(ExitCode.KILLED.getExitCode(),
containerStatus.getExitStatus());
// Assert that the process is not alive anymore
@ -300,13 +312,8 @@ public void testLocalFilesCleanup() throws InterruptedException,
ContainerLaunchContext container = recordFactory.newRecordInstance(ContainerLaunchContext.class);
// ////// Construct the Container-id
ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId.setApplicationId(appId);
appAttemptId.setAttemptId(1);
ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
cId.setAppId(appId);
cId.setAppAttemptId(appAttemptId);
ContainerId cId = createContainerId();
ApplicationId appId = cId.getApplicationAttemptId().getApplicationId();
container.setContainerId(cId);
container.setUser(user);
@ -325,7 +332,10 @@ public void testLocalFilesCleanup() throws InterruptedException,
rsrc_alpha.setType(LocalResourceType.FILE);
rsrc_alpha.setTimestamp(file.lastModified());
String destinationFile = "dest_file";
containerLaunchContext.setLocalResource(destinationFile, rsrc_alpha);
Map<String, LocalResource> localResources =
new HashMap<String, LocalResource>();
localResources.put(destinationFile, rsrc_alpha);
containerLaunchContext.setLocalResources(localResources);
containerLaunchContext.setUser(container.getUser());
containerLaunchContext.setContainerId(container.getContainerId());
containerLaunchContext.setResource(recordFactory
@ -340,7 +350,8 @@ public void testLocalFilesCleanup() throws InterruptedException,
BaseContainerManagerTest.waitForContainerState(containerManager, cId,
ContainerState.COMPLETE);
BaseContainerManagerTest.waitForApplicationState(containerManager, cId.getAppId(),
BaseContainerManagerTest.waitForApplicationState(containerManager,
cId.getApplicationAttemptId().getApplicationId(),
ApplicationState.RUNNING);
// Now ascertain that the resources are localised correctly.
@ -372,7 +383,8 @@ public void testLocalFilesCleanup() throws InterruptedException,
containerManager.handle(new CMgrCompletedAppsEvent(Arrays
.asList(new ApplicationId[] { appId })));
BaseContainerManagerTest.waitForApplicationState(containerManager, cId.getAppId(),
BaseContainerManagerTest.waitForApplicationState(containerManager,
cId.getApplicationAttemptId().getApplicationId(),
ApplicationState.FINISHED);
// Now ascertain that the resources are localised correctly.

View File

@ -418,7 +418,7 @@ private class WrappedContainer {
} else {
localResources = Collections.<String, LocalResource> emptyMap();
}
when(ctxt.getAllLocalResources()).thenReturn(localResources);
when(ctxt.getLocalResources()).thenReturn(localResources);
if (withServiceData) {
Random r = new Random();
@ -429,7 +429,7 @@ private class WrappedContainer {
} else {
serviceData = Collections.<String, ByteBuffer> emptyMap();
}
when(ctxt.getAllServiceData()).thenReturn(serviceData);
when(ctxt.getServiceData()).thenReturn(serviceData);
c = newContainer(dispatcher, ctxt);
dispatcher.start();

View File

@ -19,6 +19,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.LocalResource;
@ -48,9 +49,12 @@ static ContainerId getMockContainer(int id) {
ApplicationId appId = mock(ApplicationId.class);
when(appId.getClusterTimestamp()).thenReturn(314159265L);
when(appId.getId()).thenReturn(3);
ApplicationAttemptId appAttemptId = mock(ApplicationAttemptId.class);
when(appAttemptId.getApplicationId()).thenReturn(appId);
when(appAttemptId.getAttemptId()).thenReturn(0);
ContainerId container = mock(ContainerId.class);
when(container.getId()).thenReturn(id);
when(container.getAppId()).thenReturn(appId);
when(container.getApplicationAttemptId()).thenReturn(appAttemptId);
return container;
}

View File

@ -25,8 +25,10 @@
import java.io.IOException;
import java.io.PrintWriter;
import java.io.Writer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import junit.framework.Assert;
@ -118,8 +120,8 @@ public void testLocalFileDeletionAfterUpload() throws IOException {
BuilderUtils.newContainerId(recordFactory, application1, appAttemptId, 1);
// Simulate log-file creation
writeContainerLogs(app1LogDir, container11);
logAggregationService.handle(new LogAggregatorContainerFinishedEvent(
container11, "0"));
logAggregationService.handle(
new LogAggregatorContainerFinishedEvent(container11, 0));
logAggregationService.handle(new LogAggregatorAppFinishedEvent(
application1));
@ -192,17 +194,19 @@ public void testMultipleAppsLogAggregation() throws IOException {
application1, this.user, null,
ContainerLogsRetentionPolicy.ALL_CONTAINERS));
ApplicationAttemptId appAttemptId1 = recordFactory.newRecordInstance(ApplicationAttemptId.class);
ApplicationAttemptId appAttemptId1 =
recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId1.setApplicationId(application1);
ContainerId container11 =
BuilderUtils.newContainerId(recordFactory, application1, appAttemptId1, 1);
// Simulate log-file creation
writeContainerLogs(app1LogDir, container11);
logAggregationService.handle(new LogAggregatorContainerFinishedEvent(
container11, "0"));
logAggregationService.handle(
new LogAggregatorContainerFinishedEvent(container11, 0));
ApplicationId application2 = BuilderUtils.newApplicationId(1234, 2);
ApplicationAttemptId appAttemptId2 = recordFactory.newRecordInstance(ApplicationAttemptId.class);
ApplicationAttemptId appAttemptId2 =
recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId1.setApplicationId(application2);
File app2LogDir =
@ -214,19 +218,22 @@ public void testMultipleAppsLogAggregation() throws IOException {
ContainerId container21 =
BuilderUtils.newContainerId(recordFactory, application2, appAttemptId2, 1);
BuilderUtils.newContainerId(recordFactory, application2,
appAttemptId2, 1);
writeContainerLogs(app2LogDir, container21);
logAggregationService.handle(new LogAggregatorContainerFinishedEvent(
container21, "0"));
logAggregationService.handle(
new LogAggregatorContainerFinishedEvent(container21, 0));
ContainerId container12 =
BuilderUtils.newContainerId(recordFactory, application1, appAttemptId1, 2);
BuilderUtils.newContainerId(recordFactory, application1, appAttemptId1,
2);
writeContainerLogs(app1LogDir, container12);
logAggregationService.handle(new LogAggregatorContainerFinishedEvent(
container12, "0"));
logAggregationService.handle(
new LogAggregatorContainerFinishedEvent(container12, 0));
ApplicationId application3 = BuilderUtils.newApplicationId(1234, 3);
ApplicationAttemptId appAttemptId3 = recordFactory.newRecordInstance(ApplicationAttemptId.class);
ApplicationAttemptId appAttemptId3 =
recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId1.setApplicationId(application3);
File app3LogDir =
@ -237,28 +244,32 @@ public void testMultipleAppsLogAggregation() throws IOException {
ContainerLogsRetentionPolicy.AM_AND_FAILED_CONTAINERS_ONLY));
ContainerId container31 =
BuilderUtils.newContainerId(recordFactory, application3, appAttemptId3, 1);
BuilderUtils.newContainerId(recordFactory, application3, appAttemptId3,
1);
writeContainerLogs(app3LogDir, container31);
logAggregationService.handle(new LogAggregatorContainerFinishedEvent(
container31, "0"));
logAggregationService.handle(
new LogAggregatorContainerFinishedEvent(container31, 0));
ContainerId container32 =
BuilderUtils.newContainerId(recordFactory, application3, appAttemptId3, 2);
BuilderUtils.newContainerId(recordFactory, application3, appAttemptId3,
2);
writeContainerLogs(app3LogDir, container32);
logAggregationService.handle(new LogAggregatorContainerFinishedEvent(
container32, "1")); // Failed container
logAggregationService.handle(
new LogAggregatorContainerFinishedEvent(container32, 1)); // Failed
ContainerId container22 =
BuilderUtils.newContainerId(recordFactory, application2, appAttemptId2, 2);
BuilderUtils.newContainerId(recordFactory, application2, appAttemptId2,
2);
writeContainerLogs(app2LogDir, container22);
logAggregationService.handle(new LogAggregatorContainerFinishedEvent(
container22, "0"));
logAggregationService.handle(
new LogAggregatorContainerFinishedEvent(container22, 0));
ContainerId container33 =
BuilderUtils.newContainerId(recordFactory, application3, appAttemptId3, 3);
BuilderUtils.newContainerId(recordFactory, application3, appAttemptId3,
3);
writeContainerLogs(app3LogDir, container33);
logAggregationService.handle(new LogAggregatorContainerFinishedEvent(
container33, "0"));
logAggregationService.handle(
new LogAggregatorContainerFinishedEvent(container33, 0));
logAggregationService.handle(new LogAggregatorAppFinishedEvent(
application2));
@ -387,8 +398,15 @@ public void testLogAggregationForRealContainerLaunch() throws IOException,
// ////// Construct the Container-id
ApplicationId appId =
recordFactory.newRecordInstance(ApplicationId.class);
appId.setClusterTimestamp(0);
appId.setId(0);
ApplicationAttemptId appAttemptId =
recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId.setApplicationId(appId);
appAttemptId.setAttemptId(1);
ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
cId.setAppId(appId);
cId.setId(0);
cId.setApplicationAttemptId(appAttemptId);
containerLaunchContext.setContainerId(cId);
containerLaunchContext.setUser(this.user);
@ -404,10 +422,15 @@ public void testLogAggregationForRealContainerLaunch() throws IOException,
rsrc_alpha.setType(LocalResourceType.FILE);
rsrc_alpha.setTimestamp(scriptFile.lastModified());
String destinationFile = "dest_file";
containerLaunchContext.setLocalResource(destinationFile, rsrc_alpha);
Map<String, LocalResource> localResources =
new HashMap<String, LocalResource>();
localResources.put(destinationFile, rsrc_alpha);
containerLaunchContext.setLocalResources(localResources);
containerLaunchContext.setUser(containerLaunchContext.getUser());
containerLaunchContext.addCommand("/bin/bash");
containerLaunchContext.addCommand(scriptFile.getAbsolutePath());
List<String> commands = new ArrayList<String>();
commands.add("/bin/bash");
commands.add(scriptFile.getAbsolutePath());
containerLaunchContext.setCommands(commands);
containerLaunchContext.setResource(recordFactory
.newRecordInstance(Resource.class));
containerLaunchContext.getResource().setMemory(100 * 1024 * 1024);

View File

@ -26,6 +26,10 @@
import java.io.FileReader;
import java.io.IOException;
import java.io.PrintWriter;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
import junit.framework.Assert;
@ -192,13 +196,15 @@ public void testContainerKillOnMemoryOverflow() throws IOException,
// ////// Construct the Container-id
ApplicationId appId =
recordFactory.newRecordInstance(ApplicationId.class);
ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class);
appId.setClusterTimestamp(0);
appId.setId(0);
ApplicationAttemptId appAttemptId =
recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId.setApplicationId(appId);
appAttemptId.setAttemptId(1);
ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
cId.setAppId(appId);
cId.setId(0);
cId.setAppAttemptId(appAttemptId);
cId.setApplicationAttemptId(appAttemptId);
containerLaunchContext.setContainerId(cId);
containerLaunchContext.setUser(user);
@ -214,10 +220,15 @@ public void testContainerKillOnMemoryOverflow() throws IOException,
rsrc_alpha.setType(LocalResourceType.FILE);
rsrc_alpha.setTimestamp(scriptFile.lastModified());
String destinationFile = "dest_file";
containerLaunchContext.setLocalResource(destinationFile, rsrc_alpha);
Map<String, LocalResource> localResources =
new HashMap<String, LocalResource>();
localResources.put(destinationFile, rsrc_alpha);
containerLaunchContext.setLocalResources(localResources);
containerLaunchContext.setUser(containerLaunchContext.getUser());
containerLaunchContext.addCommand("/bin/bash");
containerLaunchContext.addCommand(scriptFile.getAbsolutePath());
List<String> commands = new ArrayList<String>();
commands.add("/bin/bash");
commands.add(scriptFile.getAbsolutePath());
containerLaunchContext.setCommands(commands);
containerLaunchContext.setResource(recordFactory
.newRecordInstance(Resource.class));
containerLaunchContext.getResource().setMemory(8 * 1024 * 1024);
@ -251,7 +262,7 @@ public void testContainerKillOnMemoryOverflow() throws IOException,
gcsRequest.setContainerId(cId);
ContainerStatus containerStatus =
containerManager.getContainerStatus(gcsRequest).getStatus();
Assert.assertEquals(String.valueOf(ExitCode.KILLED.getExitCode()),
Assert.assertEquals(ExitCode.KILLED.getExitCode(),
containerStatus.getExitStatus());
String expectedMsgPattern =
"Container \\[pid=" + pid + ",containerID=" + cId

View File

@ -112,7 +112,9 @@ public ContainerState getContainerState() {
};
nmContext.getContainers().put(containerId, container);
//TODO: Gross hack. Fix in code.
nmContext.getApplications().get(containerId.getAppId()).getContainers()
ApplicationId applicationId =
containerId.getApplicationAttemptId().getApplicationId();
nmContext.getApplications().get(applicationId).getContainers()
.put(containerId, container);
writeContainerLogs(conf, nmContext, containerId);

View File

@ -79,7 +79,8 @@ public class AMLauncher implements Runnable {
private final RMAppAttempt application;
private final Configuration conf;
private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
private final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
private final ApplicationTokenSecretManager applicationTokenSecretManager;
private final ClientToAMSecretManager clientToAMSecretManager;
private final AMLauncherEventType eventType;
@ -87,9 +88,9 @@ public class AMLauncher implements Runnable {
@SuppressWarnings("rawtypes")
private final EventHandler handler;
@SuppressWarnings("unchecked")
public AMLauncher(RMContext rmContext, RMAppAttempt application,
AMLauncherEventType eventType,ApplicationTokenSecretManager applicationTokenSecretManager,
AMLauncherEventType eventType,
ApplicationTokenSecretManager applicationTokenSecretManager,
ClientToAMSecretManager clientToAMSecretManager, Configuration conf) {
this.application = application;
this.conf = new Configuration(conf); // Just not to touch the sec-info class
@ -106,7 +107,8 @@ private void connect() throws IOException {
ContainerId masterContainerID = application.getMasterContainer().getId();
containerMgrProxy =
getContainerMgrProxy(masterContainerID.getAppId());
getContainerMgrProxy(
masterContainerID.getApplicationAttemptId().getApplicationId());
}
private void launch() throws IOException {
@ -169,12 +171,12 @@ private ContainerLaunchContext createAMContainerLaunchContext(
// Construct the actual Container
ContainerLaunchContext container = recordFactory.newRecordInstance(ContainerLaunchContext.class);
container.addAllCommands(applicationMasterContext.getCommandList());
container.setCommands(applicationMasterContext.getCommandList());
StringBuilder mergedCommand = new StringBuilder();
String failCount = Integer.toString(application.getAppAttemptId()
.getAttemptId());
List<String> commandList = new ArrayList<String>();
for (String str : container.getCommandList()) {
for (String str : container.getCommands()) {
// This is out-right wrong. AM FAIL count should be passed via env.
String result =
str.replaceFirst(ApplicationConstants.AM_FAIL_COUNT_STRING,
@ -182,21 +184,21 @@ private ContainerLaunchContext createAMContainerLaunchContext(
mergedCommand.append(result).append(" ");
commandList.add(result);
}
container.clearCommands();
container.addAllCommands(commandList);
container.setCommands(commandList);
/** add the failed count to the app master command line */
LOG.info("Command to launch container " +
containerID + " : " + mergedCommand);
container.addAllEnv(applicationMasterContext.getAllEnvironment());
container.addAllEnv(setupTokensInEnv(applicationMasterContext));
Map<String, String> environment =
applicationMasterContext.getAllEnvironment();
environment.putAll(setupTokensInEnv(applicationMasterContext));
container.setEnv(environment);
// Construct the actual Container
container.setContainerId(containerID);
container.setUser(applicationMasterContext.getUser());
container.setResource(applicationMasterContext.getMasterCapability());
container.addAllLocalResources(applicationMasterContext.getAllResourcesTodo());
container.setLocalResources(applicationMasterContext.getAllResourcesTodo());
container.setContainerTokens(applicationMasterContext.getFsTokensTodo());
return container;
}

View File

@ -37,7 +37,6 @@ public class ApplicationMasterLauncher extends AbstractService implements
private static final Log LOG = LogFactory.getLog(
ApplicationMasterLauncher.class);
private final ThreadPoolExecutor launcherPool;
private final EventHandler handler;
private LauncherThread launcherHandlingThread;
private final BlockingQueue<Runnable> masterEvents
@ -52,7 +51,6 @@ public ApplicationMasterLauncher(ApplicationTokenSecretManager
RMContext context) {
super(ApplicationMasterLauncher.class.getName());
this.context = context;
this.handler = context.getDispatcher().getEventHandler();
/* register to dispatcher */
this.context.getDispatcher().register(AMLauncherEventType.class, this);
this.launcherPool = new ThreadPoolExecutor(1, 10, 1,
@ -67,14 +65,16 @@ public void start() {
super.start();
}
protected Runnable createRunnableLauncher(RMAppAttempt application, AMLauncherEventType event) {
protected Runnable createRunnableLauncher(RMAppAttempt application,
AMLauncherEventType event) {
Runnable launcher = new AMLauncher(context, application, event,
applicationTokenSecretManager, clientToAMSecretManager, getConfig());
return launcher;
}
private void launch(RMAppAttempt application) {
Runnable launcher = createRunnableLauncher(application, AMLauncherEventType.LAUNCH);
Runnable launcher = createRunnableLauncher(application,
AMLauncherEventType.LAUNCH);
masterEvents.add(launcher);
}

View File

@ -49,9 +49,7 @@
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@ -180,7 +178,8 @@ public synchronized NodeId getNextNodeId() throws IOException {
}
private String containerPathFromContainerId(ContainerId containerId) {
String appString = ConverterUtils.toString(containerId.getAppId());
String appString = ConverterUtils.toString(
containerId.getApplicationAttemptId().getApplicationId());
return appString + "/" + containerId.getId();
}
@ -197,7 +196,10 @@ public void storeMasterContainer(Container container) throws IOException {
ContainerPBImpl containerPBImpl = (ContainerPBImpl) container;
try {
zkClient.setData(APPS + ConverterUtils.toString(container.getId().getAppId()) +
zkClient.setData(APPS +
ConverterUtils.toString(
container.getId().getApplicationAttemptId().getApplicationId())
+
ZK_PATH_SEPARATOR + APP_MASTER_CONTAINER
, containerPBImpl.getProto().toByteArray(), -1);
} catch(InterruptedException ie) {

View File

@ -215,7 +215,8 @@ synchronized public RMContainer allocate(NodeType type, SchedulerNode node,
Resources.addTo(currentConsumption, container.getResource());
if (LOG.isDebugEnabled()) {
LOG.debug("allocate: applicationId=" + container.getId().getAppId()
LOG.debug("allocate: applicationAttemptId="
+ container.getId().getApplicationAttemptId()
+ " container=" + container.getId() + " host="
+ container.getNodeId().getHost() + " type=" + type);
}

View File

@ -198,8 +198,8 @@ public synchronized void reserveResource(
}
// Cannot reserve more than one application on a given node!
if (!this.reservedContainer.getContainer().getId().getAppAttemptId().equals(
reservedContainer.getContainer().getId().getAppAttemptId())) {
if (!this.reservedContainer.getContainer().getId().getApplicationAttemptId().equals(
reservedContainer.getContainer().getId().getApplicationAttemptId())) {
throw new IllegalStateException("Trying to reserve" +
" container " + reservedContainer +
" for application " + application.getApplicationId() +
@ -221,7 +221,7 @@ public synchronized void reserveResource(
public synchronized void unreserveResource(SchedulerApp application) {
// Cannot unreserve for wrong application...
ApplicationAttemptId reservedApplication =
reservedContainer.getContainer().getId().getAppAttemptId();
reservedContainer.getContainer().getId().getApplicationAttemptId();
if (!reservedApplication.equals(
application.getApplicationAttemptId())) {
throw new IllegalStateException("Trying to unreserve " +

View File

@ -22,6 +22,7 @@
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@ -65,7 +66,8 @@ public static ContainerStatus createAbnormalContainerStatus(
recordFactory.newRecordInstance(ContainerStatus.class);
containerStatus.setContainerId(containerId);
containerStatus.setDiagnostics(diagnostics);
containerStatus.setExitStatus("ABORTED");
containerStatus.setExitStatus(
YarnConfiguration.ABORTED_CONTAINER_EXIT_STATUS);
containerStatus.setState(ContainerState.COMPLETE);
return containerStatus;
}

View File

@ -580,14 +580,15 @@ private synchronized void nodeUpdate(RMNode nm,
} else {
LOG.info("Skipping scheduling since node " + nm +
" is reserved by application " +
node.getReservedContainer().getContainerId().getAppId());
node.getReservedContainer().getContainerId().getApplicationAttemptId()
);
}
}
private void containerLaunchedOnNode(ContainerId containerId, SchedulerNode node) {
// Get the application for the finished container
ApplicationAttemptId applicationAttemptId = containerId.getAppAttemptId();
ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId();
SchedulerApp application = getApplication(applicationAttemptId);
if (application == null) {
LOG.info("Unknown application: " + applicationAttemptId +
@ -704,7 +705,7 @@ private synchronized void completedContainer(RMContainer rmContainer,
Container container = rmContainer.getContainer();
// Get the application for the finished container
ApplicationAttemptId applicationAttemptId = container.getId().getAppAttemptId();
ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId();
SchedulerApp application = getApplication(applicationAttemptId);
if (application == null) {
LOG.info("Container " + container + " of" +
@ -739,7 +740,7 @@ SchedulerNode getNode(NodeId nodeId) {
private RMContainer getRMContainer(ContainerId containerId) {
SchedulerApp application =
getApplication(containerId.getAppAttemptId());
getApplication(containerId.getApplicationAttemptId());
return (application == null) ? null : application.getRMContainer(containerId);
}

View File

@ -645,7 +645,7 @@ public void handle(SchedulerEvent event) {
private void containerLaunchedOnNode(ContainerId containerId, SchedulerNode node) {
// Get the application for the finished container
ApplicationAttemptId applicationAttemptId = containerId.getAppAttemptId();
ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId();
SchedulerApp application = getApplication(applicationAttemptId);
if (application == null) {
LOG.info("Unknown application: " + applicationAttemptId +
@ -667,7 +667,7 @@ private synchronized void containerCompleted(RMContainer rmContainer,
// Get the application for the finished container
Container container = rmContainer.getContainer();
ApplicationAttemptId applicationAttemptId = container.getId().getAppAttemptId();
ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId();
SchedulerApp application = getApplication(applicationAttemptId);
// Get the node on which the container was allocated
@ -751,7 +751,7 @@ public synchronized SchedulerNodeReport getNodeReport(NodeId nodeId) {
private RMContainer getRMContainer(ContainerId containerId) {
SchedulerApp application =
getApplication(containerId.getAppAttemptId());
getApplication(containerId.getApplicationAttemptId());
return (application == null) ? null : application.getRMContainer(containerId);
}

View File

@ -56,7 +56,7 @@ public NodeId getNodeId() {
public void containerStatus(Container container) throws Exception {
Map<ApplicationId, List<ContainerStatus>> conts =
new HashMap<ApplicationId, List<ContainerStatus>>();
conts.put(container.getId().getAppId(),
conts.put(container.getId().getApplicationAttemptId().getApplicationId(),
Arrays.asList(new ContainerStatus[] { container.getContainerStatus() }));
nodeHeartbeat(conts, true);
}

View File

@ -155,11 +155,15 @@ public void heartbeat() throws IOException {
}
@Override
synchronized public StartContainerResponse startContainer(StartContainerRequest request) throws YarnRemoteException {
ContainerLaunchContext containerLaunchContext = request.getContainerLaunchContext();
synchronized public StartContainerResponse startContainer(
StartContainerRequest request)
throws YarnRemoteException {
ContainerLaunchContext containerLaunchContext =
request.getContainerLaunchContext();
ApplicationId applicationId = containerLaunchContext.getContainerId()
.getAppId();
ApplicationId applicationId =
containerLaunchContext.getContainerId().getApplicationAttemptId().
getApplicationId();
List<Container> applicationContainers = containers.get(applicationId);
if (applicationContainers == null) {
@ -169,7 +173,8 @@ synchronized public StartContainerResponse startContainer(StartContainerRequest
// Sanity check
for (Container container : applicationContainers) {
if (container.getId().compareTo(containerLaunchContext.getContainerId()) == 0) {
if (container.getId().compareTo(containerLaunchContext.getContainerId())
== 0) {
throw new IllegalStateException(
"Container " + containerLaunchContext.getContainerId() +
" already setup on node " + containerManagerAddress);
@ -209,7 +214,8 @@ synchronized public void checkResourceUsage() {
synchronized public StopContainerResponse stopContainer(StopContainerRequest request)
throws YarnRemoteException {
ContainerId containerID = request.getContainerId();
String applicationId = String.valueOf(containerID.getAppId().getId());
String applicationId = String.valueOf(
containerID.getApplicationAttemptId().getApplicationId().getId());
// Mark the container as COMPLETE
List<Container> applicationContainers = containers.get(applicationId);
@ -259,7 +265,9 @@ synchronized public StopContainerResponse stopContainer(StopContainerRequest req
@Override
synchronized public GetContainerStatusResponse getContainerStatus(GetContainerStatusRequest request) throws YarnRemoteException {
ContainerId containerId = request.getContainerId();
List<Container> appContainers = containers.get(containerId.getAppId());
List<Container> appContainers =
containers.get(
containerId.getApplicationAttemptId().getApplicationId());
Container container = null;
for (Container c : appContainers) {
if (c.getId().equals(containerId)) {

View File

@ -27,19 +27,13 @@
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
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.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.util.Records;
import com.google.common.collect.Lists;
@ -218,10 +212,10 @@ public void handle(RMAppEvent event) {
}
public static RMApp newApplication(int i) {
final ApplicationId id = newAppID(i);
final ApplicationAttemptId appAttemptId = newAppAttemptID(newAppID(i), 0);
final Container masterContainer = Records.newRecord(Container.class);
ContainerId containerId = Records.newRecord(ContainerId.class);
containerId.setAppId(id);
containerId.setApplicationAttemptId(appAttemptId);
masterContainer.setId(containerId);
masterContainer.setNodeHttpAddress("node:port");
final String user = newUserName();
@ -233,7 +227,7 @@ public static RMApp newApplication(int i) {
return new ApplicationBase() {
@Override
public ApplicationId getApplicationId() {
return id;
return appAttemptId.getApplicationId();
}
@Override
public String getUser() {

View File

@ -154,8 +154,8 @@ public static SchedulerNode getMockNode(
public static ContainerId getMockContainerId(SchedulerApp application) {
ContainerId containerId = mock(ContainerId.class);
doReturn(application.getApplicationAttemptId()).when(containerId).getAppAttemptId();
doReturn(application.getApplicationId()).when(containerId).getAppId();
doReturn(application.getApplicationAttemptId()).
when(containerId).getApplicationAttemptId();
doReturn(application.getNewContainerId()).when(containerId).getId();
return containerId;
}

View File

@ -224,8 +224,6 @@ public AMRMProtocol run() {
RegisterApplicationMasterRequest request =
recordFactory
.newRecordInstance(RegisterApplicationMasterRequest.class);
ApplicationMaster applicationMaster = recordFactory
.newRecordInstance(ApplicationMaster.class);
request.setApplicationAttemptId(resourceManager.getRMContext()
.getRMApps().get(appID).getCurrentAppAttempt().getAppAttemptId());
scheduler.registerApplicationMaster(request);
@ -293,12 +291,13 @@ public Void run() {
.newRecordInstance(GetContainerStatusRequest.class);
ContainerId containerID =
recordFactory.newRecordInstance(ContainerId.class);
ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class);
ApplicationAttemptId appAttemptId =
recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId.setApplicationId(appID);
appAttemptId.setAttemptId(1);
containerID.setAppId(appID);
appAttemptId.setApplicationId(appID);
containerID.setApplicationAttemptId(appAttemptId);
containerID.setId(1);
containerID.setAppAttemptId(appAttemptId);
request.setContainerId(containerID);
client.getContainerStatus(request);
} catch (YarnRemoteException e) {
@ -347,9 +346,9 @@ public Void run() {
ApplicationAttemptId appAttemptId = recordFactory.newRecordInstance(ApplicationAttemptId.class);
appAttemptId.setApplicationId(appID);
appAttemptId.setAttemptId(1);
containerID.setAppId(appID);
appAttemptId.setApplicationId(appID);
containerID.setApplicationAttemptId(appAttemptId);
containerID.setId(1);
containerID.setAppAttemptId(appAttemptId);
request.setContainerId(containerID);
try {
client.getContainerStatus(request);