Merge -r 1176761:1176762 from trunk to branch-0.23 to fix MAPREDUCE-3078.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1176763 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
456e6e1de6
commit
e6f5bf6dd8
|
@ -1437,6 +1437,9 @@ Release 0.23.0 - Unreleased
|
|||
|
||||
MAPREDUCE-3110. Fixed TestRPC failure. (vinodkv)
|
||||
|
||||
MAPREDUCE-3078. Ensure MapReduce AM reports progress correctly for
|
||||
displaying on the RM Web-UI. (vinodkv via acmurthy)
|
||||
|
||||
Release 0.22.0 - Unreleased
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -55,6 +55,12 @@
|
|||
<artifactId>hadoop-yarn-server-resourcemanager</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-yarn-server-resourcemanager</artifactId>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-shuffle</artifactId>
|
||||
|
|
|
@ -549,9 +549,9 @@ public class MRAppMaster extends CompositeService {
|
|||
// It's more test friendly to put it here.
|
||||
DefaultMetricsSystem.initialize("MRAppMaster");
|
||||
|
||||
/** create a job event for job intialization */
|
||||
// create a job event for job intialization
|
||||
JobEvent initJobEvent = new JobEvent(job.getID(), JobEventType.JOB_INIT);
|
||||
/** send init to the job (this does NOT trigger job execution) */
|
||||
// Send init to the job (this does NOT trigger job execution)
|
||||
// This is a synchronous call, not an event through dispatcher. We want
|
||||
// job-init to be done completely here.
|
||||
jobEventDispatcher.handle(initJobEvent);
|
||||
|
|
|
@ -92,6 +92,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
|
|||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
||||
import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
||||
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authorize.AccessControlList;
|
||||
|
@ -584,25 +585,17 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
|
|||
public JobReport getReport() {
|
||||
readLock.lock();
|
||||
try {
|
||||
JobReport report = recordFactory.newRecordInstance(JobReport.class);
|
||||
report.setJobId(jobId);
|
||||
report.setJobState(getState());
|
||||
|
||||
// TODO - Fix to correctly setup report and to check state
|
||||
if (report.getJobState() == JobState.NEW) {
|
||||
return report;
|
||||
}
|
||||
|
||||
report.setStartTime(startTime);
|
||||
report.setFinishTime(finishTime);
|
||||
report.setSetupProgress(setupProgress);
|
||||
report.setCleanupProgress(cleanupProgress);
|
||||
report.setMapProgress(computeProgress(mapTasks));
|
||||
report.setReduceProgress(computeProgress(reduceTasks));
|
||||
report.setJobName(jobName);
|
||||
report.setUser(username);
|
||||
JobState state = getState();
|
||||
|
||||
return report;
|
||||
if (getState() == JobState.NEW) {
|
||||
return MRBuilderUtils.newJobReport(jobId, jobName, username, state,
|
||||
startTime, finishTime, setupProgress, 0.0f,
|
||||
0.0f, cleanupProgress);
|
||||
}
|
||||
|
||||
return MRBuilderUtils.newJobReport(jobId, jobName, username, state,
|
||||
startTime, finishTime, setupProgress, computeProgress(mapTasks),
|
||||
computeProgress(reduceTasks), cleanupProgress);
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
|
||||
package org.apache.hadoop.mapreduce.v2.app.local;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -30,15 +31,19 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent;
|
|||
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerRequestEvent;
|
||||
import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||
import org.apache.hadoop.yarn.api.records.AMResponse;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.Container;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
/**
|
||||
|
@ -65,6 +70,20 @@ public class LocalContainerAllocator extends RMCommunicator
|
|||
this.appID = context.getApplicationID();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected synchronized void heartbeat() throws Exception {
|
||||
AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest(
|
||||
this.applicationAttemptId, this.lastResponseID, super
|
||||
.getApplicationProgress(), new ArrayList<ResourceRequest>(),
|
||||
new ArrayList<ContainerId>());
|
||||
AllocateResponse allocateResponse = scheduler.allocate(allocateRequest);
|
||||
AMResponse response = allocateResponse.getAMResponse();
|
||||
if (response.getReboot()) {
|
||||
// TODO
|
||||
LOG.info("Event from RM: shutting down Application Master");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handle(ContainerAllocatorEvent event) {
|
||||
if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.hadoop.mapreduce.v2.app.rm;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.security.PrivilegedAction;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -29,6 +28,7 @@ import org.apache.hadoop.mapreduce.JobID;
|
|||
import org.apache.hadoop.mapreduce.MRJobConfig;
|
||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
|
||||
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
||||
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
|
||||
|
@ -42,17 +42,12 @@ import org.apache.hadoop.security.token.TokenIdentifier;
|
|||
import org.apache.hadoop.yarn.YarnException;
|
||||
import org.apache.hadoop.yarn.api.AMRMProtocol;
|
||||
import org.apache.hadoop.yarn.api.ApplicationConstants;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
||||
import org.apache.hadoop.yarn.api.records.AMResponse;
|
||||
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.Resource;
|
||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
|
@ -64,7 +59,7 @@ import org.apache.hadoop.yarn.service.AbstractService;
|
|||
/**
|
||||
* Registers/unregisters to RM and sends heartbeats to RM.
|
||||
*/
|
||||
public class RMCommunicator extends AbstractService {
|
||||
public abstract class RMCommunicator extends AbstractService {
|
||||
private static final Log LOG = LogFactory.getLog(RMContainerAllocator.class);
|
||||
private int rmPollInterval;//millis
|
||||
protected ApplicationId applicationId;
|
||||
|
@ -74,7 +69,7 @@ public class RMCommunicator extends AbstractService {
|
|||
protected EventHandler eventHandler;
|
||||
protected AMRMProtocol scheduler;
|
||||
private final ClientService clientService;
|
||||
private int lastResponseID;
|
||||
protected int lastResponseID;
|
||||
private Resource minContainerCapability;
|
||||
private Resource maxContainerCapability;
|
||||
|
||||
|
@ -121,6 +116,34 @@ public class RMCommunicator extends AbstractService {
|
|||
return job;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the appProgress. Can be used only after this component is started.
|
||||
* @return the appProgress.
|
||||
*/
|
||||
protected float getApplicationProgress() {
|
||||
// For now just a single job. In future when we have a DAG, we need an
|
||||
// aggregate progress.
|
||||
JobReport report = this.job.getReport();
|
||||
float setupWeight = 0.05f;
|
||||
float cleanupWeight = 0.05f;
|
||||
float mapWeight = 0.0f;
|
||||
float reduceWeight = 0.0f;
|
||||
int numMaps = this.job.getTotalMaps();
|
||||
int numReduces = this.job.getTotalReduces();
|
||||
if (numMaps == 0 && numReduces == 0) {
|
||||
} else if (numMaps == 0) {
|
||||
reduceWeight = 0.9f;
|
||||
} else if (numReduces == 0) {
|
||||
mapWeight = 0.9f;
|
||||
} else {
|
||||
mapWeight = reduceWeight = 0.45f;
|
||||
}
|
||||
return (report.getSetupProgress() * setupWeight
|
||||
+ report.getCleanupProgress() * cleanupWeight
|
||||
+ report.getMapProgress() * mapWeight + report.getReduceProgress()
|
||||
* reduceWeight);
|
||||
}
|
||||
|
||||
protected void register() {
|
||||
//Register
|
||||
String host =
|
||||
|
@ -262,18 +285,5 @@ public class RMCommunicator extends AbstractService {
|
|||
});
|
||||
}
|
||||
|
||||
protected synchronized void heartbeat() throws Exception {
|
||||
AllocateRequest allocateRequest =
|
||||
recordFactory.newRecordInstance(AllocateRequest.class);
|
||||
allocateRequest.setApplicationAttemptId(applicationAttemptId);
|
||||
allocateRequest.setResponseId(lastResponseID);
|
||||
allocateRequest.addAllAsks(new ArrayList<ResourceRequest>());
|
||||
allocateRequest.addAllReleases(new ArrayList<ContainerId>());
|
||||
AllocateResponse allocateResponse = scheduler.allocate(allocateRequest);
|
||||
AMResponse response = allocateResponse.getAMResponse();
|
||||
if (response.getReboot()) {
|
||||
LOG.info("Event from RM: shutting down Application Master");
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract void heartbeat() throws Exception;
|
||||
}
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|||
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.util.BuilderUtils;
|
||||
|
||||
/**
|
||||
* Keeps the data structures to send container requests to RM.
|
||||
|
@ -107,15 +108,11 @@ public abstract class RMContainerRequestor extends RMCommunicator {
|
|||
LOG.info("maxTaskFailuresPerNode is " + maxTaskFailuresPerNode);
|
||||
}
|
||||
|
||||
protected abstract void heartbeat() throws Exception;
|
||||
|
||||
protected AMResponse makeRemoteRequest() throws YarnRemoteException {
|
||||
AllocateRequest allocateRequest = recordFactory
|
||||
.newRecordInstance(AllocateRequest.class);
|
||||
allocateRequest.setApplicationAttemptId(applicationAttemptId);
|
||||
allocateRequest.setResponseId(lastResponseID);
|
||||
allocateRequest.addAllAsks(new ArrayList<ResourceRequest>(ask));
|
||||
allocateRequest.addAllReleases(new ArrayList<ContainerId>(release));
|
||||
AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest(
|
||||
applicationAttemptId, lastResponseID, super.getApplicationProgress(),
|
||||
new ArrayList<ResourceRequest>(ask), new ArrayList<ContainerId>(
|
||||
release));
|
||||
AllocateResponse allocateResponse = scheduler.allocate(allocateRequest);
|
||||
AMResponse response = allocateResponse.getAMResponse();
|
||||
lastResponseID = response.getResponseId();
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -19,27 +19,25 @@
|
|||
package org.apache.hadoop.mapreduce.v2.util;
|
||||
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
public class MRBuilderUtils {
|
||||
|
||||
private static final RecordFactory recordFactory = RecordFactoryProvider
|
||||
.getRecordFactory(null);
|
||||
|
||||
public static JobId newJobId(ApplicationId appId, int id) {
|
||||
JobId jobId = recordFactory.newRecordInstance(JobId.class);
|
||||
JobId jobId = Records.newRecord(JobId.class);
|
||||
jobId.setAppId(appId);
|
||||
jobId.setId(id);
|
||||
return jobId;
|
||||
}
|
||||
|
||||
public static TaskId newTaskId(JobId jobId, int id, TaskType taskType) {
|
||||
TaskId taskId = recordFactory.newRecordInstance(TaskId.class);
|
||||
TaskId taskId = Records.newRecord(TaskId.class);
|
||||
taskId.setJobId(jobId);
|
||||
taskId.setId(id);
|
||||
taskId.setTaskType(taskType);
|
||||
|
@ -48,9 +46,27 @@ public class MRBuilderUtils {
|
|||
|
||||
public static TaskAttemptId newTaskAttemptId(TaskId taskId, int attemptId) {
|
||||
TaskAttemptId taskAttemptId =
|
||||
recordFactory.newRecordInstance(TaskAttemptId.class);
|
||||
Records.newRecord(TaskAttemptId.class);
|
||||
taskAttemptId.setTaskId(taskId);
|
||||
taskAttemptId.setId(attemptId);
|
||||
return taskAttemptId;
|
||||
}
|
||||
|
||||
public static JobReport newJobReport(JobId jobId, String jobName,
|
||||
String userName, JobState state, long startTime, long finishTime,
|
||||
float setupProgress, float mapProgress, float reduceProgress,
|
||||
float cleanupProgress) {
|
||||
JobReport report = Records.newRecord(JobReport.class);
|
||||
report.setJobId(jobId);
|
||||
report.setJobName(jobName);
|
||||
report.setUser(userName);
|
||||
report.setJobState(state);
|
||||
report.setStartTime(startTime);
|
||||
report.setFinishTime(finishTime);
|
||||
report.setSetupProgress(setupProgress);
|
||||
report.setCleanupProgress(cleanupProgress);
|
||||
report.setMapProgress(mapProgress);
|
||||
report.setReduceProgress(reduceProgress);
|
||||
return report;
|
||||
}
|
||||
}
|
|
@ -64,6 +64,12 @@
|
|||
<artifactId>hadoop-yarn-server-resourcemanager</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-yarn-server-resourcemanager</artifactId>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-yarn-server-common</artifactId>
|
||||
|
|
|
@ -88,6 +88,12 @@
|
|||
<artifactId>hadoop-yarn-server-resourcemanager</artifactId>
|
||||
<version>${yarn.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-yarn-server-resourcemanager</artifactId>
|
||||
<version>${yarn.version}</version>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-core</artifactId>
|
||||
|
|
|
@ -20,7 +20,9 @@ package org.apache.hadoop.yarn.util;
|
|||
|
||||
import java.net.URI;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
||||
|
@ -184,6 +186,13 @@ public class BuilderUtils {
|
|||
return id;
|
||||
}
|
||||
|
||||
public static NodeId newNodeId(String host, int port) {
|
||||
NodeId nodeId = recordFactory.newRecordInstance(NodeId.class);
|
||||
nodeId.setHost(host);
|
||||
nodeId.setPort(port);
|
||||
return nodeId;
|
||||
}
|
||||
|
||||
public static Container newContainer(RecordFactory recordFactory,
|
||||
ApplicationAttemptId appAttemptId, int containerId, NodeId nodeId,
|
||||
String nodeHttpAddress, Resource resource, Priority priority) {
|
||||
|
@ -266,5 +275,18 @@ public class BuilderUtils {
|
|||
url.setFile(file);
|
||||
return url;
|
||||
}
|
||||
|
||||
|
||||
public static AllocateRequest newAllocateRequest(
|
||||
ApplicationAttemptId applicationAttemptId, int responseID,
|
||||
float appProgress, List<ResourceRequest> resourceAsk,
|
||||
List<ContainerId> containersToBeReleased) {
|
||||
AllocateRequest allocateRequest = recordFactory
|
||||
.newRecordInstance(AllocateRequest.class);
|
||||
allocateRequest.setApplicationAttemptId(applicationAttemptId);
|
||||
allocateRequest.setResponseId(responseID);
|
||||
allocateRequest.setProgress(appProgress);
|
||||
allocateRequest.addAllAsks(resourceAsk);
|
||||
allocateRequest.addAllReleases(containersToBeReleased);
|
||||
return allocateRequest;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,6 +37,20 @@
|
|||
|
||||
<build>
|
||||
<plugins>
|
||||
|
||||
<!-- Publish tests jar -->
|
||||
<plugin>
|
||||
<artifactId>maven-jar-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<goals>
|
||||
<goal>test-jar</goal>
|
||||
</goals>
|
||||
<phase>test-compile</phase>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
|
||||
<plugin>
|
||||
<artifactId>maven-antrun-plugin</artifactId>
|
||||
<executions>
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|||
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.ipc.RPCUtil;
|
||||
import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
|
||||
|
@ -250,13 +251,10 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent> {
|
|||
|
||||
if (rmContext.getRMApps().putIfAbsent(applicationId, application) !=
|
||||
null) {
|
||||
LOG.info("Application with id " + applicationId +
|
||||
" is already present! Cannot add a duplicate!");
|
||||
// don't send event through dispatcher as it will be handled by app
|
||||
// already present with this id.
|
||||
application.handle(new RMAppRejectedEvent(applicationId,
|
||||
"Application with this id is already present! " +
|
||||
"Cannot add a duplicate!"));
|
||||
String message = "Application with id " + applicationId
|
||||
+ " is already present! Cannot add a duplicate!";
|
||||
LOG.info(message);
|
||||
throw RPCUtil.getRemoteException(message);
|
||||
} else {
|
||||
this.rmContext.getDispatcher().getEventHandler().handle(
|
||||
new RMAppEvent(applicationId, RMAppEventType.START));
|
||||
|
|
|
@ -98,7 +98,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|||
private ContainerAllocationExpirer containerAllocationExpirer;
|
||||
protected NMLivelinessMonitor nmLivelinessMonitor;
|
||||
protected NodesListManager nodesListManager;
|
||||
private SchedulerEventDispatcher schedulerDispatcher;
|
||||
private EventHandler<SchedulerEvent> schedulerDispatcher;
|
||||
protected RMAppManager rmAppManager;
|
||||
|
||||
private WebApp webApp;
|
||||
|
@ -119,7 +119,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|||
@Override
|
||||
public synchronized void init(Configuration conf) {
|
||||
|
||||
this.rmDispatcher = new AsyncDispatcher();
|
||||
this.rmDispatcher = createDispatcher();
|
||||
addIfService(this.rmDispatcher);
|
||||
|
||||
this.containerAllocationExpirer = new ContainerAllocationExpirer(
|
||||
|
@ -138,8 +138,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|||
this.conf = new YarnConfiguration(conf);
|
||||
// Initialize the scheduler
|
||||
this.scheduler = createScheduler();
|
||||
this.schedulerDispatcher = new SchedulerEventDispatcher(this.scheduler);
|
||||
addService(this.schedulerDispatcher);
|
||||
this.schedulerDispatcher = createSchedulerEventDispatcher();
|
||||
addIfService(this.schedulerDispatcher);
|
||||
this.rmDispatcher.register(SchedulerEventType.class,
|
||||
this.schedulerDispatcher);
|
||||
|
||||
|
@ -195,6 +195,14 @@ public class ResourceManager extends CompositeService implements Recoverable {
|
|||
super.init(conf);
|
||||
}
|
||||
|
||||
protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
|
||||
return new SchedulerEventDispatcher(this.scheduler);
|
||||
}
|
||||
|
||||
protected Dispatcher createDispatcher() {
|
||||
return new AsyncDispatcher();
|
||||
}
|
||||
|
||||
protected void addIfService(Object object) {
|
||||
if (object instanceof Service) {
|
||||
addService((Service) object);
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
|||
* look at {@link RMAppImpl} for its implementation. This interface
|
||||
* exposes methods to access various updates in application status/report.
|
||||
*/
|
||||
public interface RMApp extends EventHandler<RMAppEvent>{
|
||||
public interface RMApp extends EventHandler<RMAppEvent> {
|
||||
|
||||
/**
|
||||
* The application id for this {@link RMApp}.
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|||
* {@link YarnConfiguration#RM_AM_MAX_RETRIES}. For specific
|
||||
* implementation take a look at {@link RMAppAttemptImpl}.
|
||||
*/
|
||||
public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent>{
|
||||
public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent> {
|
||||
|
||||
/**
|
||||
* Get the application attempt id for this {@link RMAppAttempt}.
|
||||
|
|
|
@ -685,6 +685,8 @@ public class RMAppAttemptImpl implements RMAppAttempt {
|
|||
public void transition(RMAppAttemptImpl appAttempt,
|
||||
RMAppAttemptEvent event) {
|
||||
|
||||
appAttempt.progress = 1.0f;
|
||||
|
||||
// Tell the app and the scheduler
|
||||
super.transition(appAttempt, event);
|
||||
|
||||
|
|
|
@ -207,13 +207,18 @@ public class SchedulerApp {
|
|||
.getDispatcher().getEventHandler(), this.rmContext
|
||||
.getContainerAllocationExpirer());
|
||||
|
||||
// Add it to allContainers list.
|
||||
newlyAllocatedContainers.add(rmContainer);
|
||||
liveContainers.put(container.getId(), rmContainer);
|
||||
|
||||
// Update consumption and track allocations
|
||||
|
||||
appSchedulingInfo.allocate(type, node, priority, request, container);
|
||||
Resources.addTo(currentConsumption, container.getResource());
|
||||
|
||||
// Inform the container
|
||||
rmContainer.handle(
|
||||
new RMContainerEvent(container.getId(), RMContainerEventType.START));
|
||||
|
||||
Resources.addTo(currentConsumption, container.getResource());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("allocate: applicationAttemptId="
|
||||
+ container.getId().getApplicationAttemptId()
|
||||
|
@ -223,12 +228,6 @@ public class SchedulerApp {
|
|||
RMAuditLogger.logSuccess(getUser(),
|
||||
AuditConstants.ALLOC_CONTAINER, "SchedulerApp",
|
||||
getApplicationId(), container.getId());
|
||||
|
||||
// Add it to allContainers list.
|
||||
newlyAllocatedContainers.add(rmContainer);
|
||||
liveContainers.put(container.getId(), rmContainer);
|
||||
|
||||
appSchedulingInfo.allocate(type, node, priority, request, container);
|
||||
|
||||
return rmContainer;
|
||||
}
|
||||
|
|
|
@ -19,10 +19,7 @@
|
|||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.event;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.Container;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
||||
|
||||
|
|
|
@ -291,7 +291,7 @@ public class FifoScheduler implements ResourceScheduler {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
private synchronized void addApplication(ApplicationAttemptId appAttemptId,
|
||||
String queueName, String user) {
|
||||
String user) {
|
||||
// TODO: Fix store
|
||||
SchedulerApp schedulerApp =
|
||||
new SchedulerApp(appAttemptId, user, DEFAULT_QUEUE,
|
||||
|
@ -628,7 +628,7 @@ public class FifoScheduler implements ResourceScheduler {
|
|||
{
|
||||
AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
|
||||
addApplication(appAddedEvent.getApplicationAttemptId(), appAddedEvent
|
||||
.getQueue(), appAddedEvent.getUser());
|
||||
.getUser());
|
||||
}
|
||||
break;
|
||||
case APP_REMOVED:
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
|
||||
public class MockAM {
|
||||
|
@ -128,7 +129,7 @@ public class MockAM {
|
|||
req.setHostName(resource);
|
||||
req.setNumContainers(containers);
|
||||
Priority pri = Records.newRecord(Priority.class);
|
||||
pri.setPriority(1);
|
||||
pri.setPriority(priority);
|
||||
req.setPriority(pri);
|
||||
Resource capability = Records.newRecord(Resource.class);
|
||||
capability.setMemory(memory);
|
||||
|
@ -139,11 +140,8 @@ public class MockAM {
|
|||
public AMResponse allocate(
|
||||
List<ResourceRequest> resourceRequest, List<ContainerId> releases)
|
||||
throws Exception {
|
||||
AllocateRequest req = Records.newRecord(AllocateRequest.class);
|
||||
req.setResponseId(++responseId);
|
||||
req.setApplicationAttemptId(attemptId);
|
||||
req.addAllAsks(resourceRequest);
|
||||
req.addAllReleases(releases);
|
||||
AllocateRequest req = BuilderUtils.newAllocateRequest(attemptId,
|
||||
++responseId, 0F, resourceRequest, releases);
|
||||
AllocateResponse resp = amRMProtocol.allocate(req);
|
||||
return resp.getAMResponse();
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -77,13 +78,14 @@ public class TestAMRMRPCResponseId {
|
|||
|
||||
am.registerAppAttempt();
|
||||
|
||||
AllocateRequest allocateRequest = recordFactory.newRecordInstance(AllocateRequest.class);
|
||||
allocateRequest.setApplicationAttemptId(attempt.getAppAttemptId());
|
||||
AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest(attempt
|
||||
.getAppAttemptId(), 0, 0F, null, null);
|
||||
|
||||
AMResponse response = amService.allocate(allocateRequest).getAMResponse();
|
||||
Assert.assertEquals(1, response.getResponseId());
|
||||
Assert.assertFalse(response.getReboot());
|
||||
allocateRequest.setResponseId(response.getResponseId());
|
||||
allocateRequest = BuilderUtils.newAllocateRequest(attempt
|
||||
.getAppAttemptId(), response.getResponseId(), 0F, null, null);
|
||||
|
||||
response = amService.allocate(allocateRequest).getAMResponse();
|
||||
Assert.assertEquals(2, response.getResponseId());
|
||||
|
@ -91,8 +93,9 @@ public class TestAMRMRPCResponseId {
|
|||
response = amService.allocate(allocateRequest).getAMResponse();
|
||||
Assert.assertEquals(2, response.getResponseId());
|
||||
|
||||
/** try sending old **/
|
||||
allocateRequest.setResponseId(0);
|
||||
/** try sending old request again **/
|
||||
allocateRequest = BuilderUtils.newAllocateRequest(attempt
|
||||
.getAppAttemptId(), 0, 0F, null, null);
|
||||
response = amService.allocate(allocateRequest).getAMResponse();
|
||||
Assert.assertTrue(response.getReboot());
|
||||
}
|
||||
|
|
|
@ -162,6 +162,7 @@ public class MockRMApp implements RMApp {
|
|||
this.diagnostics = new StringBuilder(diag);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handle(RMAppEvent event) {
|
||||
}
|
||||
|
||||
|
|
|
@ -83,6 +83,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
||||
import org.apache.hadoop.yarn.util.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -240,12 +241,8 @@ public class TestContainerTokenSecretManager {
|
|||
ask.add(rr);
|
||||
ArrayList<ContainerId> release = new ArrayList<ContainerId>();
|
||||
|
||||
AllocateRequest allocateRequest =
|
||||
recordFactory.newRecordInstance(AllocateRequest.class);
|
||||
allocateRequest.setApplicationAttemptId(appAttempt.getAppAttemptId());
|
||||
allocateRequest.setResponseId(0);
|
||||
allocateRequest.addAllAsks(ask);
|
||||
allocateRequest.addAllReleases(release);
|
||||
AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest(
|
||||
appAttempt.getAppAttemptId(), 0, 0F, ask, release);
|
||||
List<Container> allocatedContainers = scheduler.allocate(allocateRequest)
|
||||
.getAMResponse().getAllocatedContainers();
|
||||
|
||||
|
|
Loading…
Reference in New Issue