|
@@ -22,11 +22,15 @@ import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.List;
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationMaster;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationState;
|
|
@@ -37,9 +41,9 @@ import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
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.server.resourcemanager.RMConfig;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.RMContext;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager.events.ASMEvent;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager.events.ApplicationFinishEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager.events.ApplicationMasterEvents.AMLauncherEventType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager.events.ApplicationMasterEvents.ApplicationEventType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager.events.ApplicationMasterEvents.ApplicationTrackerEventType;
|
|
@@ -59,7 +63,8 @@ import org.apache.hadoop.yarn.state.StateMachineFactory;
|
|
|
*/
|
|
|
@Private
|
|
|
@Unstable
|
|
|
-public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<ApplicationEventType>> {
|
|
|
+public class ApplicationMasterInfo implements AppContext,
|
|
|
+ EventHandler<ApplicationMasterInfoEvent> {
|
|
|
private static final Log LOG = LogFactory.getLog(ApplicationMasterInfo.class);
|
|
|
private final ApplicationSubmissionContext submissionContext;
|
|
|
private ApplicationMaster master;
|
|
@@ -74,7 +79,11 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
private static String DIAGNOSTIC_KILL_APPLICATION = "Application was killed.";
|
|
|
private static String DIAGNOSTIC_AM_FAILED = "Application Master failed";
|
|
|
private static String DIAGNOSTIC_AM_LAUNCH_FAILED = "Application Master failed to launch";
|
|
|
-
|
|
|
+
|
|
|
+ private final int amMaxRetries;
|
|
|
+ private final AMLivelinessMonitor amLivelinessMonitor;
|
|
|
+ private final ReadLock readLock;
|
|
|
+ private final WriteLock writeLock;
|
|
|
private int numFailed = 0;
|
|
|
private final ApplicationStore appStore;
|
|
|
|
|
@@ -93,15 +102,13 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
private final LaunchedTransition launchedTransition = new LaunchedTransition();
|
|
|
private final FailedLaunchTransition failedLaunchTransition = new FailedLaunchTransition();
|
|
|
|
|
|
- private final StateMachine<ApplicationState, ApplicationEventType,
|
|
|
- ASMEvent<ApplicationEventType>> stateMachine;
|
|
|
+ private final StateMachine<ApplicationState,
|
|
|
+ ApplicationEventType, ApplicationMasterInfoEvent> stateMachine;
|
|
|
|
|
|
- private final StateMachineFactory<ApplicationMasterInfo,
|
|
|
- ApplicationState, ApplicationEventType, ASMEvent<ApplicationEventType>> stateMachineFactory
|
|
|
-
|
|
|
- = new StateMachineFactory
|
|
|
- <ApplicationMasterInfo, ApplicationState, ApplicationEventType, ASMEvent<ApplicationEventType>>
|
|
|
- (ApplicationState.PENDING)
|
|
|
+ private final StateMachineFactory<ApplicationMasterInfo, ApplicationState,
|
|
|
+ ApplicationEventType, ApplicationMasterInfoEvent> stateMachineFactory
|
|
|
+ = new StateMachineFactory<ApplicationMasterInfo, ApplicationState,
|
|
|
+ ApplicationEventType, ApplicationMasterInfoEvent>(ApplicationState.PENDING)
|
|
|
|
|
|
// Transitions from PENDING State
|
|
|
.addTransition(ApplicationState.PENDING, ApplicationState.ALLOCATING,
|
|
@@ -147,8 +154,6 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
// Transitions from LAUNCHED State
|
|
|
.addTransition(ApplicationState.LAUNCHED, ApplicationState.CLEANUP,
|
|
|
ApplicationEventType.KILL, killTransition)
|
|
|
- .addTransition(ApplicationState.LAUNCHED, ApplicationState.FAILED,
|
|
|
- ApplicationEventType.EXPIRE, expireTransition)
|
|
|
.addTransition(ApplicationState.LAUNCHED, ApplicationState.RUNNING,
|
|
|
ApplicationEventType.REGISTERED, new RegisterTransition())
|
|
|
.addTransition(ApplicationState.LAUNCHED, ApplicationState.LAUNCHED,
|
|
@@ -189,8 +194,7 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
|
|
|
// Transitions from COMPLETED State
|
|
|
.addTransition(ApplicationState.COMPLETED, ApplicationState.COMPLETED,
|
|
|
- EnumSet.of(ApplicationEventType.EXPIRE,
|
|
|
- ApplicationEventType.FINISH, ApplicationEventType.KILL,
|
|
|
+ EnumSet.of(ApplicationEventType.FINISH, ApplicationEventType.KILL,
|
|
|
ApplicationEventType.RECOVER))
|
|
|
|
|
|
// Transitions from FAILED State
|
|
@@ -209,8 +213,10 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
|
|
|
|
|
|
|
|
|
- public ApplicationMasterInfo(RMContext context, String user,
|
|
|
- ApplicationSubmissionContext submissionContext, String clientToken, ApplicationStore appStore) {
|
|
|
+ public ApplicationMasterInfo(RMContext context, Configuration conf,
|
|
|
+ String user, ApplicationSubmissionContext submissionContext,
|
|
|
+ String clientToken, ApplicationStore appStore,
|
|
|
+ AMLivelinessMonitor amLivelinessMonitor) {
|
|
|
this.user = user;
|
|
|
this.handler = context.getDispatcher().getEventHandler();
|
|
|
this.syncHandler = context.getDispatcher().getSyncHandler();
|
|
@@ -228,6 +234,14 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
master.setDiagnostics("");
|
|
|
this.appStore = appStore;
|
|
|
this.startTime = System.currentTimeMillis();
|
|
|
+ this.amMaxRetries = conf.getInt(RMConfig.AM_MAX_RETRIES,
|
|
|
+ RMConfig.DEFAULT_AM_MAX_RETRIES);
|
|
|
+ LOG.info("AM max retries: " + this.amMaxRetries);
|
|
|
+ this.amLivelinessMonitor = amLivelinessMonitor;
|
|
|
+
|
|
|
+ ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock();
|
|
|
+ this.readLock = readWriteLock.readLock();
|
|
|
+ this.writeLock = readWriteLock.writeLock();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -307,12 +321,13 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
}
|
|
|
|
|
|
/* the applicaiton master completed successfully */
|
|
|
- private static class DoneTransition implements
|
|
|
- MultipleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>, ApplicationState> {
|
|
|
+ private static class DoneTransition
|
|
|
+ implements
|
|
|
+ MultipleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent, ApplicationState> {
|
|
|
|
|
|
@Override
|
|
|
public ApplicationState transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
masterInfo.handler.handle(new ASMEvent<SNEventType>(
|
|
|
SNEventType.CLEANUP, masterInfo));
|
|
|
masterInfo.handler.handle(new ASMEvent<AMLauncherEventType>(
|
|
@@ -320,26 +335,29 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
masterInfo.handler.handle(new ASMEvent<ApplicationTrackerEventType>(
|
|
|
ApplicationTrackerEventType.REMOVE, masterInfo));
|
|
|
masterInfo.finishTime = System.currentTimeMillis();
|
|
|
+
|
|
|
+ masterInfo.amLivelinessMonitor.unRegister(event.getApplicationId());
|
|
|
+
|
|
|
ApplicationFinishEvent finishEvent = (ApplicationFinishEvent) event;
|
|
|
return finishEvent.getFinalApplicationState();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static class AllocatingKillTransition implements
|
|
|
- SingleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ private static class AllocatingKillTransition implements
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
masterInfo.handler.handle(new ASMEvent<ApplicationTrackerEventType>(ApplicationTrackerEventType.REMOVE,
|
|
|
masterInfo));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private static class KillTransition implements
|
|
|
- SingleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
masterInfo.finishTime = System.currentTimeMillis();
|
|
|
masterInfo.getMaster().setDiagnostics(DIAGNOSTIC_KILL_APPLICATION);
|
|
|
masterInfo.handler.handle(new ASMEvent<SNEventType>(SNEventType.CLEANUP, masterInfo));
|
|
@@ -349,12 +367,12 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static class RecoverLaunchTransition implements SingleArcTransition
|
|
|
- <ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ private static class RecoverLaunchTransition implements
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
masterInfo.syncHandler.handle(new ASMEvent<ApplicationTrackerEventType>(
|
|
|
ApplicationTrackerEventType.ADD, masterInfo));
|
|
|
|
|
@@ -363,11 +381,11 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static class FailedLaunchTransition implements
|
|
|
- SingleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ private static class FailedLaunchTransition implements
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
masterInfo.finishTime = System.currentTimeMillis();
|
|
|
masterInfo.getMaster().setDiagnostics(DIAGNOSTIC_AM_LAUNCH_FAILED);
|
|
|
masterInfo.handler.handle(new ASMEvent<SNEventType>(
|
|
@@ -376,74 +394,82 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
}
|
|
|
|
|
|
private static class LaunchTransition implements
|
|
|
- SingleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
masterInfo.handler.handle(new ASMEvent<AMLauncherEventType>(
|
|
|
AMLauncherEventType.LAUNCH, masterInfo));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private static class RecoverRunningTransition implements
|
|
|
- SingleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
masterInfo.syncHandler.handle(new ASMEvent<ApplicationTrackerEventType>(
|
|
|
ApplicationTrackerEventType.ADD, masterInfo));
|
|
|
/* make sure the time stamp is update else expiry thread will expire this */
|
|
|
- masterInfo.master.getStatus().setLastSeen(System.currentTimeMillis());
|
|
|
+ masterInfo.amLivelinessMonitor.receivedPing(event.getApplicationId());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private static class RecoverLaunchedTransition implements
|
|
|
- SingleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
masterInfo.syncHandler.handle(new ASMEvent<ApplicationTrackerEventType>(
|
|
|
ApplicationTrackerEventType.ADD, masterInfo));
|
|
|
|
|
|
- /* make sure the time stamp is update else expiry thread will expire this */
|
|
|
- masterInfo.master.getStatus().setLastSeen(System.currentTimeMillis());
|
|
|
+ masterInfo.amLivelinessMonitor.register(event.getApplicationId());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
|
|
|
private static class LaunchedTransition implements
|
|
|
- SingleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
- /* make sure the time stamp is update else expiry thread will expire this */
|
|
|
- masterInfo.master.getStatus().setLastSeen(System.currentTimeMillis());
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
+ masterInfo.amLivelinessMonitor.register(event.getApplicationId());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static class ExpireTransition implements
|
|
|
- SingleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ private static class ExpireTransition implements
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
/* for now this is the same as killed transition but will change later */
|
|
|
masterInfo.handler.handle(new ASMEvent<SNEventType>(SNEventType.CLEANUP,
|
|
|
masterInfo));
|
|
|
masterInfo.handler.handle(new ASMEvent<AMLauncherEventType>(
|
|
|
AMLauncherEventType.CLEANUP, masterInfo));
|
|
|
masterInfo.handler.handle(new ASMEvent<ApplicationTrackerEventType>(
|
|
|
- ApplicationTrackerEventType.EXPIRE, masterInfo));
|
|
|
+ ApplicationTrackerEventType.EXPIRE, masterInfo));
|
|
|
masterInfo.numFailed++;
|
|
|
+
|
|
|
+ /* check to see if the number of retries are reached or not */
|
|
|
+ if (masterInfo.getFailedCount() < masterInfo.amMaxRetries) {
|
|
|
+ masterInfo.handler.handle(new ApplicationMasterInfoEvent(
|
|
|
+ ApplicationEventType.ALLOCATE, event.getApplicationId()));
|
|
|
+ } else {
|
|
|
+ masterInfo.handler.handle(new ApplicationMasterInfoEvent(
|
|
|
+ ApplicationEventType.FAILED_MAX_RETRIES, masterInfo
|
|
|
+ .getApplicationID()));
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
|
|
|
|
/* Transition to schedule again on a container launch failure for AM */
|
|
|
private static class ScheduleTransition implements
|
|
|
- SingleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
masterInfo.masterContainer = null;
|
|
|
/* schedule for a slot */
|
|
|
masterInfo.handler.handle(new ASMEvent<SNEventType>(SNEventType.SCHEDULE,
|
|
@@ -452,50 +478,61 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
}
|
|
|
|
|
|
/* Transition to start the process of allocating for the AM container */
|
|
|
- private static class AllocateTransition implements
|
|
|
- SingleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ private static class AllocateTransition implements
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
/* notify tracking applications that an applicaiton has been added */
|
|
|
- masterInfo.handler.handle(new ASMEvent<ApplicationTrackerEventType>(
|
|
|
+ // TODO: For now, changing to synchHandler. Instead we should use register/deregister.
|
|
|
+ masterInfo.syncHandler.handle(new ASMEvent<ApplicationTrackerEventType>(
|
|
|
ApplicationTrackerEventType.ADD, masterInfo));
|
|
|
|
|
|
/* schedule for a slot */
|
|
|
- masterInfo.handler.handle(new ASMEvent<SNEventType>(SNEventType.SCHEDULE,
|
|
|
- masterInfo));
|
|
|
+ masterInfo.handler.handle(new ASMEvent<SNEventType>(
|
|
|
+ SNEventType.SCHEDULE, masterInfo));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/* Transition on a container allocated for a container */
|
|
|
- private static class AllocatedTransition implements SingleArcTransition<ApplicationMasterInfo,
|
|
|
- ASMEvent<ApplicationEventType>> {
|
|
|
+ private static class AllocatedTransition
|
|
|
+ implements
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
/* set the container that was generated by the scheduler negotiator */
|
|
|
- masterInfo.masterContainer = event.getAppContext().getMasterContainer();
|
|
|
+ ApplicationMasterAllocatedEvent allocatedEvent =
|
|
|
+ (ApplicationMasterAllocatedEvent) event;
|
|
|
+ masterInfo.masterContainer = allocatedEvent.getMasterContainer();
|
|
|
try {
|
|
|
masterInfo.appStore.storeMasterContainer(masterInfo.masterContainer);
|
|
|
} catch(IOException ie) {
|
|
|
//TODO ignore for now fix later.
|
|
|
}
|
|
|
+
|
|
|
+ /* we need to launch the applicaiton master on allocated transition */
|
|
|
+ masterInfo.handler.handle(new ApplicationMasterInfoEvent(
|
|
|
+ ApplicationEventType.LAUNCH, masterInfo.getApplicationID()));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static class RegisterTransition implements SingleArcTransition<ApplicationMasterInfo,
|
|
|
- ASMEvent<ApplicationEventType>> {
|
|
|
+ private static class RegisterTransition implements
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
- ApplicationMaster registeredMaster = event.getAppContext().getMaster();
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
+ ApplicationMasterRegistrationEvent registrationEvent =
|
|
|
+ (ApplicationMasterRegistrationEvent) event;
|
|
|
+ ApplicationMaster registeredMaster = registrationEvent
|
|
|
+ .getApplicationMaster();
|
|
|
masterInfo.master.setHost(registeredMaster.getHost());
|
|
|
masterInfo.master.setTrackingUrl(registeredMaster.getTrackingUrl());
|
|
|
masterInfo.master.setRpcPort(registeredMaster.getRpcPort());
|
|
|
masterInfo.master.setStatus(registeredMaster.getStatus());
|
|
|
masterInfo.master.getStatus().setProgress(0.0f);
|
|
|
- masterInfo.master.getStatus().setLastSeen(System.currentTimeMillis());
|
|
|
+ masterInfo.amLivelinessMonitor.receivedPing(event.getApplicationId());
|
|
|
try {
|
|
|
masterInfo.appStore.updateApplicationState(masterInfo.master);
|
|
|
} catch(IOException ie) {
|
|
@@ -506,52 +543,61 @@ public class ApplicationMasterInfo implements AppContext, EventHandler<ASMEvent<
|
|
|
|
|
|
/* transition to finishing state on a cleanup, for now its not used, but will need it
|
|
|
* later */
|
|
|
- private static class FailedTransition implements
|
|
|
- SingleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ private static class FailedTransition implements
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
LOG.info("Failed application: " + masterInfo.getApplicationID());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
|
|
|
/* Just a status update transition */
|
|
|
- private static class StatusUpdateTransition implements
|
|
|
- SingleArcTransition<ApplicationMasterInfo, ASMEvent<ApplicationEventType>> {
|
|
|
+ private static class StatusUpdateTransition implements
|
|
|
+ SingleArcTransition<ApplicationMasterInfo, ApplicationMasterInfoEvent> {
|
|
|
|
|
|
@Override
|
|
|
public void transition(ApplicationMasterInfo masterInfo,
|
|
|
- ASMEvent<ApplicationEventType> event) {
|
|
|
- masterInfo.master.setStatus(event.getAppContext().getStatus());
|
|
|
- masterInfo.master.getStatus().setLastSeen(System.currentTimeMillis());
|
|
|
+ ApplicationMasterInfoEvent event) {
|
|
|
+ ApplicationMasterStatusUpdateEvent statusUpdateEvent =
|
|
|
+ (ApplicationMasterStatusUpdateEvent) event;
|
|
|
+ masterInfo.master.setStatus(statusUpdateEvent.getApplicationStatus());
|
|
|
+ masterInfo.amLivelinessMonitor.receivedPing(event.getApplicationId());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized void handle(ASMEvent<ApplicationEventType> event) {
|
|
|
- ApplicationId appID = event.getAppContext().getApplicationID();
|
|
|
- LOG.info("Processing event for " + appID + " of type " + event.getType());
|
|
|
- final ApplicationState oldState = getState();
|
|
|
- try {
|
|
|
- /* keep the master in sync with the state machine */
|
|
|
- stateMachine.doTransition(event.getType(), event);
|
|
|
- master.setState(stateMachine.getCurrentState());
|
|
|
- LOG.info("State is " + stateMachine.getCurrentState());
|
|
|
- } catch (InvalidStateTransitonException e) {
|
|
|
- LOG.error("Can't handle this event at current state", e);
|
|
|
- /* TODO fail the application on the failed transition */
|
|
|
- }
|
|
|
+ public synchronized void handle(ApplicationMasterInfoEvent event) {
|
|
|
+
|
|
|
+ this.writeLock.lock();
|
|
|
+
|
|
|
try {
|
|
|
- appStore.updateApplicationState(master);
|
|
|
- } catch(IOException ie) {
|
|
|
- //TODO ignore for now
|
|
|
- }
|
|
|
- if (oldState != getState()) {
|
|
|
- LOG.info(appID + " State change from "
|
|
|
- + oldState + " to "
|
|
|
- + getState());
|
|
|
+ ApplicationId appID = event.getApplicationId();
|
|
|
+ LOG.info("Processing event for " + appID + " of type "
|
|
|
+ + event.getType());
|
|
|
+ final ApplicationState oldState = getState();
|
|
|
+ try {
|
|
|
+ /* keep the master in sync with the state machine */
|
|
|
+ stateMachine.doTransition(event.getType(), event);
|
|
|
+ master.setState(stateMachine.getCurrentState());
|
|
|
+ LOG.info("State is " + stateMachine.getCurrentState());
|
|
|
+ } catch (InvalidStateTransitonException e) {
|
|
|
+ LOG.error("Can't handle this event at current state", e);
|
|
|
+ /* TODO fail the application on the failed transition */
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ appStore.updateApplicationState(master);
|
|
|
+ } catch (IOException ie) {
|
|
|
+ // TODO ignore for now
|
|
|
+ }
|
|
|
+ if (oldState != getState()) {
|
|
|
+ LOG.info(appID + " State change from " + oldState + " to "
|
|
|
+ + getState());
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ this.writeLock.unlock();
|
|
|
}
|
|
|
}
|
|
|
}
|