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);
next = dynamicState.withState(MachineState.KILL) .withPendingLocalization(dynamicState.newAssignment, pendingDownload); break; next = dynamicState.withState(MachineState.KILL_BLOB_UPDATE); break; next = dynamicState.withState(MachineState.KILL_AND_RELAUNCH).withProfileActions(mod, Collections.emptySet()); break;
staticState.localState); return dynamicState .withCurrentAssignment(c, dynamicState.pendingChangingBlobsAssignment).withState(MachineState.WAITING_FOR_WORKER_START) .withPendingChangingBlobs(Collections.emptySet(), null); } catch (TimeoutException ex) {
long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000; if (hbAgeMs <= staticState.hbTimeoutMs) { return dynamicState.withState(MachineState.RUNNING);
/** * 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; } }
return dynamicState.withCurrentAssignment(c, dynamicState.pendingLocalization).withState(MachineState.WAITING_FOR_WORKER_START).withPendingLocalization(null, null); } catch (TimeoutException e) {
/** * 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()); }
/** * 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); }
/** * 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); }
long hbAgeMs = (Time.currentTimeSecs() - hb.get_time_secs()) * 1000; if (hbAgeMs <= staticState.hbTimeoutMs) { return dynamicState.withState(MachineState.RUNNING);
/** * 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. */ private 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; }
/** * 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 */ private 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.requestDownloadTopologyBlobs(dynamicState.newAssignment, staticState.port, staticState.changingCallback); return dynamicState.withPendingLocalization(dynamicState.newAssignment, pendingDownload) .withState(MachineState.WAITING_FOR_BLOB_LOCALIZATION); }