if (!forSameTopology(rc.assignment, dynamicState.currentAssignment) && !forSameTopology(rc.assignment, dynamicState.newAssignment)) { stateMachineStep(dynamicState.withNewAssignment(newAssignment.get()) .withProfileActions(origProfileActions, dynamicState.pendingStopProfileActions) .withChangingBlobs(changingResourcesToHandle), staticState); LOG.info("SLOT {}: Changing current assignment from {} to {}", staticState.port, dynamicState.currentAssignment, nextState.currentAssignment); saveNewAssignment(nextState.currentAssignment); if (equivalent(nextState.newAssignment, nextState.currentAssignment) && nextState.currentAssignment != null && nextState.currentAssignment.get_owner() == null && nextState.newAssignment != null && nextState.newAssignment.get_owner() != null) { saveNewAssignment(nextState.newAssignment); nextState = nextState.withCurrentAssignment(nextState.container, nextState.newAssignment);
static DynamicState stateMachineStep(DynamicState dynamicState, StaticState staticState) throws Exception { LOG.debug("STATE {}", dynamicState.state); switch (dynamicState.state) { case EMPTY: return handleEmpty(dynamicState, staticState); case RUNNING: return handleRunning(dynamicState, staticState); case WAITING_FOR_WORKER_START: return handleWaitingForWorkerStart(dynamicState, staticState); case KILL_BLOB_UPDATE: return handleKillBlobUpdate(dynamicState, staticState); case KILL_AND_RELAUNCH: return handleKillAndRelaunch(dynamicState, staticState); case KILL: return handleKill(dynamicState, staticState); case WAITING_FOR_BLOB_LOCALIZATION: return handleWaitingForBlobLocalization(dynamicState, staticState); case WAITING_FOR_BLOB_UPDATE: return handleWaitingForBlobUpdate(dynamicState, staticState); default: throw new IllegalStateException("Code not ready to handle a state of " + dynamicState.state); } }
private Slot mkSlot(int port) throws Exception { return new Slot(localizer, superConf, launcher, host, port, localState, stormClusterState, iSuper, cachedAssignments, metricsExec, metricsProcessor, slotMetrics); }
static DynamicState handleEmpty(DynamicState dynamicState, StaticState staticState) throws InterruptedException, IOException { assert dynamicState.changingBlobs.isEmpty(); assert dynamicState.pendingChangingBlobsAssignment == null; if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) { return prepareForNewAssignmentNoWorkersRunning(dynamicState, staticState); } dynamicState = updateAssignmentIfNeeded(dynamicState); //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.emptySet(), Collections.emptySet()); } //Drop the change notifications we are not running anything right now dynamicState = drainAllChangingBlobs(dynamicState); Time.sleep(1000); return dynamicState; }
if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) { return killContainerFor(KillReason.ASSIGNMENT_CHANGED, dynamicState, staticState); dynamicState = updateAssignmentIfNeeded(dynamicState); LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs); return killContainerFor(KillReason.HB_TIMEOUT, dynamicState, staticState); dynamicState = filterChangingBlobsFor(dynamicState, dynamicState.currentAssignment); if (!dynamicState.changingBlobs.isEmpty()) { return killContainerFor(KillReason.BLOB_CHANGED, dynamicState, staticState);
static DynamicState stateMachineStep(DynamicState dynamicState, StaticState staticState) throws Exception { LOG.debug("STATE {}", dynamicState.state); switch (dynamicState.state) { case EMPTY: return handleEmpty(dynamicState, staticState); case RUNNING: return handleRunning(dynamicState, staticState); case WAITING_FOR_WORKER_START: return handleWaitingForWorkerStart(dynamicState, staticState); case KILL_AND_RELAUNCH: return handleKillAndRelaunch(dynamicState, staticState); case KILL: return handleKill(dynamicState, staticState); case WAITING_FOR_BASIC_LOCALIZATION: return handleWaitingForBasicLocalization(dynamicState, staticState); case WAITING_FOR_BLOB_LOCALIZATION: return handleWaitingForBlobLocalization(dynamicState, staticState); default: throw new IllegalStateException("Code not ready to handle a state of "+dynamicState.state); } }
dynamicState = filterChangingBlobsFor(dynamicState, dynamicState.pendingLocalization); if (!dynamicState.changingBlobs.isEmpty()) { dynamicState = informChangedBlobs(dynamicState, dynamicState.pendingLocalization); if (!equivalent(dynamicState.newAssignment, dynamicState.pendingLocalization)) { return prepareForNewAssignmentNoWorkersRunning(dynamicState.withPendingChangingBlobs(Collections.emptySet(), null), staticState); dynamicState = updateAssignmentIfNeeded(dynamicState); staticState.slotMetrics.numWorkersLaunched.mark(); Container c =
slot = mkSlot(port); slots.put(port, slot); slot.start(); slot.setNewAssignment(allAssignments.get(port)); slot.addProfilerActions(filtered.get(port));
stateMachineStep(dynamicState.withNewAssignment(newAssignment.get()) .withProfileActions(origProfileActions, dynamicState.pendingStopProfileActions), staticState); if (!equivalent(nextState.currentAssignment, dynamicState.currentAssignment)) { LOG.info("SLOT {}: Changing current assignment from {} to {}", staticState.port, dynamicState.currentAssignment, nextState.currentAssignment); saveNewAssignment(nextState.currentAssignment); if (equivalent(nextState.newAssignment, nextState.currentAssignment) && nextState.currentAssignment != null && nextState.currentAssignment.get_owner() == null && nextState.newAssignment != null && nextState.newAssignment.get_owner() != null) { saveNewAssignment(nextState.newAssignment); nextState = nextState.withCurrentAssignment(nextState.container, nextState.newAssignment);
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; }
if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) { return killContainerForChangedAssignment(dynamicState, staticState); if (timeDiffms > staticState.firstHbTimeoutMs) { LOG.warn("SLOT {}: Container {} failed to launch in {} ms.", staticState.port, dynamicState.container, staticState.firstHbTimeoutMs); return killAndRelaunchContainer(dynamicState, staticState);
DynamicState dynamicState = new DynamicState(assignment, container, assignment, slotMetrics); DynamicState nextState = Slot.stateMachineStep(dynamicState, staticState); assertEquals(MachineState.KILL_AND_RELAUNCH, nextState.state); verify(container).kill(); assertTrue(Time.currentTimeMillis() > 1000); nextState = Slot.stateMachineStep(nextState, staticState); assertEquals(MachineState.KILL_AND_RELAUNCH, nextState.state); verify(container).forceKill(); assertTrue(Time.currentTimeMillis() > 2000); nextState = Slot.stateMachineStep(nextState, staticState); assertEquals(MachineState.WAITING_FOR_WORKER_START, nextState.state); verify(container).relaunch(); nextState = Slot.stateMachineStep(nextState, staticState); assertEquals(MachineState.WAITING_FOR_WORKER_START, nextState.state); assertTrue(Time.currentTimeMillis() > 3000); nextState = Slot.stateMachineStep(nextState, staticState); assertEquals(MachineState.RUNNING, nextState.state);
public synchronized void shutdownAllWorkers(UniFunc<Slot> onWarnTimeout, UniFunc<Slot> onErrorTimeout) { for (Slot slot : slots.values()) { LOG.info("Setting {} assignment to null", slot); slot.setNewAssignment(null); Exception exp = null; for (Slot slot : slots.values()) { LOG.info("Waiting for {} to be EMPTY, currently {}", slot, slot.getMachineState()); try { while (slot.getMachineState() != MachineState.EMPTY) { long timeSpentMillis = Time.currentTimeMillis() - startTime; if (timeSpentMillis > ERROR_MILLIS) {
Collection<String> detachedRunningWorkers = SupervisorUtils.supervisorWorkerIds(superConf); for (Slot slot : slots.values()) { String workerId = slot.getWorkerId(); slot.start();
/** * Informs the async localizer for all of blobs that the worker acknowledged the change of blobs. * Worker has stop as of now. * * PRECONDITION: container is null * PRECONDITION: changingBlobs should only be for the given assignment. * @param dynamicState the current state * @return the futures for the current assignment. */ private static DynamicState informChangedBlobs(DynamicState dynamicState, LocalAssignment assignment) { assert dynamicState.container == null; assert dynamicState.changingBlobs.stream().allMatch((cr) -> forSameTopology(cr.assignment, assignment)); Set<Future<Void>> futures = new HashSet<>(dynamicState.changingBlobs.size()); // We need to add the new futures to the existing ones if (forSameTopology(dynamicState.pendingChangingBlobsAssignment, assignment)) { futures.addAll(dynamicState.pendingChangingBlobs); } // Acknowledge all changing blobs as futures for (BlobChanging rc : dynamicState.changingBlobs) { futures.add(rc.latch.countDown()); } LOG.debug("found changing blobs {} moving them to pending...", dynamicState.changingBlobs); return dynamicState.withChangingBlobs(Collections.emptySet()) .withPendingChangingBlobs(futures, assignment); }
@Test public void testEquivilant() { LocalAssignment a = mkLocalAssignment("A", mkExecutorInfoList(1, 2, 3, 4, 5), mkWorkerResources(100.0, 100.0, 100.0)); LocalAssignment aResized = mkLocalAssignment("A", mkExecutorInfoList(1, 2, 3, 4, 5), mkWorkerResources(100.0, 200.0, 100.0)); LocalAssignment b = mkLocalAssignment("B", mkExecutorInfoList(1, 2, 3, 4, 5, 6), mkWorkerResources(100.0, 100.0, 100.0)); LocalAssignment bReordered = mkLocalAssignment("B", mkExecutorInfoList(6, 5, 4, 3, 2, 1), mkWorkerResources(100.0, 100.0, 100.0)); assertTrue(Slot.equivalent(null, null)); assertTrue(Slot.equivalent(a, a)); assertTrue(Slot.equivalent(b, bReordered)); assertTrue(Slot.equivalent(bReordered, b)); assertFalse(Slot.equivalent(a, aResized)); assertFalse(Slot.equivalent(aResized, a)); assertFalse(Slot.equivalent(a, null)); assertFalse(Slot.equivalent(null, b)); assertFalse(Slot.equivalent(a, b)); }
@Test public void testEmptyToEmpty() throws Exception { try (SimulatedTime t = new SimulatedTime(1010)) { AsyncLocalizer localizer = mock(AsyncLocalizer.class); LocalState state = mock(LocalState.class); BlobChangingCallback cb = mock(BlobChangingCallback.class); ContainerLauncher containerLauncher = mock(ContainerLauncher.class); ISupervisor iSuper = mock(ISupervisor.class); SlotMetrics slotMetrics = new SlotMetrics(new StormMetricsRegistry()); StaticState staticState = new StaticState(localizer, 1000, 1000, 1000, 1000, containerLauncher, "localhost", 8080, iSuper, state, cb, null, null, slotMetrics); DynamicState dynamicState = new DynamicState(null, null, null, slotMetrics); DynamicState nextState = Slot.handleEmpty(dynamicState, staticState); assertEquals(MachineState.EMPTY, nextState.state); assertTrue(Time.currentTimeMillis() > 1000); } }
@Override public void close() { for (Slot slot : slots.values()) { try { slot.close(); } catch (Exception e) { LOG.error("Error trying to shutdown {}", slot, e); } } }
/** * State Transitions for KILL state. * PRECONDITION: container.kill() was called * PRECONDITION: container != null && currentAssignment != null * @param dynamicState current state * @param staticState static data * @return the next state * @throws Exception on any error */ private static DynamicState handleKill(DynamicState dynamicState, StaticState staticState) throws Exception { assert (dynamicState.container != null); assert (dynamicState.currentAssignment != null); if (dynamicState.container.areAllProcessesDead()) { LOG.info("SLOT {} all processes are dead...", staticState.port); return cleanupCurrentContainer(dynamicState, staticState, dynamicState.pendingLocalization == null ? MachineState.EMPTY : MachineState.WAITING_FOR_BLOB_LOCALIZATION); } LOG.warn("SLOT {} force kill and wait...", staticState.port); dynamicState.container.forceKill(); Time.sleep(staticState.killSleepMs); return dynamicState; }
assert !dynamicState.pendingChangingBlobs.isEmpty(); if (!equivalent(dynamicState.newAssignment, dynamicState.currentAssignment)) { return prepareForNewAssignmentNoWorkersRunning(dynamicState.withCurrentAssignment(null, null) .withPendingChangingBlobs(Collections.emptySet(), null), staticState); dynamicState = filterChangingBlobsFor(dynamicState, dynamicState.pendingChangingBlobsAssignment); if (!dynamicState.changingBlobs.isEmpty()) { dynamicState = informChangedBlobs(dynamicState, dynamicState.pendingChangingBlobsAssignment);