this.dynamicState = new DynamicState(currentAssignment, container, this.newAssignment.get(), slotMetrics); if (MachineState.RUNNING == dynamicState.state) {
stateMachineStep(dynamicState.withNewAssignment(newAssignment.get()) .withProfileActions(origProfileActions, dynamicState.pendingStopProfileActions) .withChangingBlobs(changingResourcesToHandle), staticState); nextState = nextState.withCurrentAssignment(nextState.container, nextState.newAssignment);
return prepareForNewAssignmentNoWorkersRunning(dynamicState.withPendingChangingBlobs(Collections.emptySet(), null), staticState); LOG.info("There are pending changes, waiting for them to finish before launching container..."); return dynamicState.withState(MachineState.WAITING_FOR_BLOB_UPDATE) .withPendingLocalization(null, null); staticState.containerLauncher.launchContainer(staticState.port, dynamicState.pendingLocalization, staticState.localState); return dynamicState .withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START) .withPendingLocalization(null, null); } catch (TimeoutException e) { return dynamicState.withState(MachineState.EMPTY);
dynamicState = dynamicState.withProfileActions(mod, modPending);
dynamicState = new DynamicState(currentAssignment, container, newAssignment); staticState = new StaticState(localizer, Utils.getInt(conf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS)) * 1000,
stateMachineStep(dynamicState.withNewAssignment(newAssignment.get()) .withProfileActions(origProfileActions, dynamicState.pendingStopProfileActions), staticState); nextState = nextState.withCurrentAssignment(nextState.container, nextState.newAssignment);
long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000; if (hbAgeMs <= staticState.hbTimeoutMs) { return dynamicState.withState(MachineState.RUNNING);
return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null); } catch (TimeoutException e) {
/** * State Transitions for WAITING_FOR_BASIC_LOCALIZATION state. * PRECONDITION: neither pendingLocalization nor pendingDownload is null. * PRECONDITION: The slot should be empty * @param dynamicState current state * @param staticState static data * @return the next state * @throws Exception on any error */ static DynamicState handleWaitingForBasicLocalization(DynamicState dynamicState, StaticState staticState) throws Exception { assert(dynamicState.pendingLocalization != null); assert(dynamicState.pendingDownload != null); assert(dynamicState.container == null); //Ignore changes to scheduling while downloading the topology code // We don't support canceling the download through the future yet, // so to keep everything in sync, just wait try { dynamicState.pendingDownload.get(1000, TimeUnit.MILLISECONDS); Future<Void> pendingDownload = staticState.localizer.requestDownloadTopologyBlobs(dynamicState.pendingLocalization, staticState.port); return dynamicState.withPendingLocalization(pendingDownload).withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION); } catch (TimeoutException e) { return dynamicState; } }
/** * Clean up a container * PRECONDITION: All of the processes have died. * @param dynamicState current state * @param staticState static data * @param nextState the next MachineState to go to. * @return the next state. */ static DynamicState cleanupCurrentContainer(DynamicState dynamicState, StaticState staticState, MachineState nextState) throws Exception { assert(dynamicState.container != null); assert(dynamicState.currentAssignment != null); assert(dynamicState.container.areAllProcessesDead()); dynamicState.container.cleanUp(); staticState.localizer.releaseSlotFor(dynamicState.currentAssignment, staticState.port); DynamicState ret = dynamicState.withCurrentAssignment(null, null); if (nextState != null) { ret = ret.withState(nextState); } return ret; }
/** * Kill the current container and relaunch it. (Something odd happened) * PRECONDITION: container != null * @param dynamicState current state * @param staticState static data * @return the next state * @throws Exception */ static DynamicState killAndRelaunchContainer(DynamicState dynamicState, StaticState staticState) throws Exception { assert(dynamicState.container != null); dynamicState.container.kill(); Time.sleep(staticState.killSleepMs); //any stop profile actions that hadn't timed out yet, we should restart after the worker is running again. HashSet<TopoProfileAction> mod = new HashSet<>(dynamicState.profileActions); mod.addAll(dynamicState.pendingStopProfileActions); return dynamicState.withState(MachineState.KILL_AND_RELAUNCH).withProfileActions(mod, Collections.<TopoProfileAction> emptySet()); }
/** * Kill the current container and start downloading what the new assignment needs, if there is a new assignment * PRECONDITION: container != null * @param dynamicState current state * @param staticState static data * @return the next state * @throws Exception */ static DynamicState killContainerForChangedAssignment(DynamicState dynamicState, StaticState staticState) throws Exception { assert(dynamicState.container != null); staticState.iSupervisor.killedWorker(staticState.port); dynamicState.container.kill(); Future<Void> pendingDownload = null; if (dynamicState.newAssignment != null) { pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment, staticState.port); } Time.sleep(staticState.killSleepMs); return dynamicState.withPendingLocalization(dynamicState.newAssignment, pendingDownload).withState(MachineState.KILL); }
/** * Prepare for a new assignment by downloading new required blobs, or going to empty if there is nothing to download. * PRECONDITION: The slot should be empty * @param dynamicState current state * @param staticState static data * @return the next state * @throws IOException on any error */ static DynamicState prepareForNewAssignmentNoWorkersRunning(DynamicState dynamicState, StaticState staticState) throws IOException { assert(dynamicState.container == null); if (dynamicState.newAssignment == null) { return dynamicState.withState(MachineState.EMPTY); } Future<Void> pendingDownload = staticState.localizer.requestDownloadBaseTopologyBlobs(dynamicState.newAssignment, staticState.port); return dynamicState.withPendingLocalization(dynamicState.newAssignment, pendingDownload).withState(MachineState.WAITING_FOR_BASIC_LOCALIZATION); }
/** * Set the new assignment for the state. This should never be called from within the state machine. * It is an input from outside. * @param newAssignment the new assignment to set * @return the updated DynamicState. */ public DynamicState withNewAssignment(LocalAssignment newAssignment) { return new DynamicState(this.state, newAssignment, this.container, this.currentAssignment, this.pendingLocalization, this.startTime, this.pendingDownload, this.profileActions, this.pendingStopProfileActions); }
static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticState) throws InterruptedException, IOException { if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) { return prepareForNewAssignmentNoWorkersRunning(dynamicState, staticState); } //Both assignments are null, just wait if (dynamicState.profileActions != null && !dynamicState.profileActions.isEmpty()) { //Nothing is scheduled here so throw away all of the profileActions LOG.warn("Dropping {} no topology is running", dynamicState.profileActions); dynamicState = dynamicState.withProfileActions(Collections.<TopoProfileAction> emptySet(), Collections.<TopoProfileAction> emptySet()); } Time.sleep(1000); return dynamicState; }
public DynamicState withPendingLocalization(Future<Void> pendingDownload) { return withPendingLocalization(this.pendingLocalization, pendingDownload); }
public DynamicState withState(final MachineState state) { long newStartTime = Time.currentTimeMillis(); return new DynamicState(state, this.newAssignment, this.container, this.currentAssignment, this.pendingLocalization, newStartTime, this.pendingDownload, this.profileActions, this.pendingStopProfileActions); }
public DynamicState withPendingLocalization(LocalAssignment pendingLocalization, Future<Void> pendingDownload) { return new DynamicState(this.state, this.newAssignment, this.container, this.currentAssignment, pendingLocalization, this.startTime, pendingDownload, this.profileActions, this.pendingStopProfileActions); }
public DynamicState withCurrentAssignment(final Container container, final LocalAssignment currentAssignment) { return new DynamicState(this.state, this.newAssignment, container, currentAssignment, this.pendingLocalization, this.startTime, this.pendingDownload, this.profileActions, this.pendingStopProfileActions); }
public DynamicState withProfileActions(Set<TopoProfileAction> profileActions, Set<TopoProfileAction> pendingStopProfileActions) { return new DynamicState(this.state, this.newAssignment, this.container, this.currentAssignment, this.pendingLocalization, this.startTime, this.pendingDownload, profileActions, pendingStopProfileActions); } };