|
@@ -23,19 +23,23 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.mapreduce.JobCounter;
|
|
|
+import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
|
|
|
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.RMCommunicator;
|
|
|
+import org.apache.hadoop.yarn.YarnException;
|
|
|
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;
|
|
@@ -57,8 +61,10 @@ public class LocalContainerAllocator extends RMCommunicator
|
|
|
LogFactory.getLog(LocalContainerAllocator.class);
|
|
|
|
|
|
private final EventHandler eventHandler;
|
|
|
- private final ApplicationId appID;
|
|
|
+// private final ApplicationId appID;
|
|
|
private AtomicInteger containerCount = new AtomicInteger();
|
|
|
+ private long retryInterval;
|
|
|
+ private long retrystartTime;
|
|
|
|
|
|
private final RecordFactory recordFactory =
|
|
|
RecordFactoryProvider.getRecordFactory(null);
|
|
@@ -67,7 +73,19 @@ public class LocalContainerAllocator extends RMCommunicator
|
|
|
AppContext context) {
|
|
|
super(clientService, context);
|
|
|
this.eventHandler = context.getEventHandler();
|
|
|
- this.appID = context.getApplicationID();
|
|
|
+// this.appID = context.getApplicationID();
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void init(Configuration conf) {
|
|
|
+ super.init(conf);
|
|
|
+ retryInterval =
|
|
|
+ getConfig().getLong(MRJobConfig.MR_AM_TO_RM_WAIT_INTERVAL_MS,
|
|
|
+ MRJobConfig.DEFAULT_MR_AM_TO_RM_WAIT_INTERVAL_MS);
|
|
|
+ // Init startTime to current time. If all goes well, it will be reset after
|
|
|
+ // first attempt to contact RM.
|
|
|
+ retrystartTime = System.currentTimeMillis();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -77,10 +95,32 @@ public class LocalContainerAllocator extends RMCommunicator
|
|
|
.getApplicationProgress(), new ArrayList<ResourceRequest>(),
|
|
|
new ArrayList<ContainerId>());
|
|
|
AllocateResponse allocateResponse = scheduler.allocate(allocateRequest);
|
|
|
- AMResponse response = allocateResponse.getAMResponse();
|
|
|
+ AMResponse response;
|
|
|
+ try {
|
|
|
+ response = allocateResponse.getAMResponse();
|
|
|
+ // Reset retry count if no exception occurred.
|
|
|
+ retrystartTime = System.currentTimeMillis();
|
|
|
+ } catch (Exception e) {
|
|
|
+ // This can happen when the connection to the RM has gone down. Keep
|
|
|
+ // re-trying until the retryInterval has expired.
|
|
|
+ if (System.currentTimeMillis() - retrystartTime >= retryInterval) {
|
|
|
+ eventHandler.handle(new JobEvent(this.getJob().getID(),
|
|
|
+ JobEventType.INTERNAL_ERROR));
|
|
|
+ throw new YarnException("Could not contact RM after " +
|
|
|
+ retryInterval + " milliseconds.");
|
|
|
+ }
|
|
|
+ // Throw this up to the caller, which may decide to ignore it and
|
|
|
+ // continue to attempt to contact the RM.
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
if (response.getReboot()) {
|
|
|
- // TODO
|
|
|
LOG.info("Event from RM: shutting down Application Master");
|
|
|
+ // This can happen if the RM has been restarted. If it is in that state,
|
|
|
+ // this application must clean itself up.
|
|
|
+ eventHandler.handle(new JobEvent(this.getJob().getID(),
|
|
|
+ JobEventType.INTERNAL_ERROR));
|
|
|
+ throw new YarnException("Resource Manager doesn't recognize AttemptId: " +
|
|
|
+ this.getContext().getApplicationID());
|
|
|
}
|
|
|
}
|
|
|
|