public TaskStateMachine(TaskId taskId, Executor executor) { this.taskId = requireNonNull(taskId, "taskId is null"); taskState = new StateMachine<>("task " + taskId, executor, TaskState.RUNNING, TERMINAL_TASK_STATES); taskState.addStateChangeListener(new StateChangeListener<TaskState>() { @Override public void stateChanged(TaskState newState) { log.debug("Task %s is %s", taskId, newState); } }); }
/** * Sets the state if the current state satisfies the specified predicate. * If the new state does not {@code .equals()} the current state, listeners and waiters will be notified. * * @return true if the state is set */ public boolean setIf(T newState, Predicate<T> predicate) { checkState(!Thread.holdsLock(lock), "Can not set state while holding the lock"); requireNonNull(newState, "newState is null"); while (true) { // check if the current state passes the predicate T currentState = get(); // change to same state is not a change, and does not notify the notify listeners if (currentState.equals(newState)) { return false; } // do not call predicate while holding the lock if (!predicate.test(currentState)) { return false; } // if state did not change while, checking the predicate, apply the new state if (compareAndSet(currentState, newState)) { return true; } } }
public ListenableFuture<TaskState> getStateChange(TaskState currentState) { requireNonNull(currentState, "currentState is null"); checkArgument(!currentState.isDone(), "Current state is already done"); ListenableFuture<TaskState> future = taskState.getStateChange(currentState); TaskState state = taskState.get(); if (state.isDone()) { return immediateFuture(state); } return future; }
private void assertStateChange(StateMachine<State> stateMachine, StateChanger stateChange, State expectedState) throws Exception State initialState = stateMachine.get(); Future<State> futureChange = stateMachine.getStateChange(initialState); stateMachine.addStateChangeListener(listenerChange::set); stateMachine.waitForStateChange(initialState, new Duration(10, SECONDS)); return stateMachine.get(); assertEquals(stateMachine.get(), expectedState); boolean isTerminalState = stateMachine.isTerminalState(expectedState); if (isTerminalState) { assertEquals(stateMachine.getStateChangeListeners(), ImmutableSet.of());
new StateMachine<>("test", executor, null); fail("expected a NullPointerException"); StateMachine<State> stateMachine = new StateMachine<>("test", executor, State.BREAKFAST); stateMachine.set(null); fail("expected a NullPointerException"); stateMachine.compareAndSet(State.BREAKFAST, null); fail("expected a NullPointerException"); stateMachine.compareAndSet(State.LUNCH, null); fail("expected a NullPointerException"); stateMachine.setIf(null, currentState -> true); fail("expected a NullPointerException"); stateMachine.setIf(null, currentState -> false); fail("expected a NullPointerException");
private static void assertStateChange(StateMachine<State> stateMachine, StateChanger stateChange, State expectedState) throws Exception { State initialState = stateMachine.get(); ListenableFuture<State> futureChange = stateMachine.getStateChange(initialState); SettableFuture<State> listenerChange = addTestListener(stateMachine); stateChange.run(); assertEquals(stateMachine.get(), expectedState); assertEquals(futureChange.get(10, SECONDS), expectedState); assertEquals(listenerChange.get(10, SECONDS), expectedState); // listeners should not be retained if we are in a terminal state boolean isTerminalState = stateMachine.isTerminalState(expectedState); if (isTerminalState) { assertEquals(stateMachine.getStateChangeListeners(), ImmutableSet.of()); } }
public synchronized boolean transitionToSchedulingSplits() { return stageState.setIf(SCHEDULING_SPLITS, currentState -> currentState == PLANNED || currentState == SCHEDULING); }
@Test public void testSet() throws Exception { StateMachine<State> stateMachine = new StateMachine<>("test", executor, State.BREAKFAST, ImmutableSet.of(State.DINNER)); assertEquals(stateMachine.get(), State.BREAKFAST); assertNoStateChange(stateMachine, () -> assertEquals(stateMachine.set(State.BREAKFAST), State.BREAKFAST)); assertStateChange(stateMachine, () -> assertEquals(stateMachine.set(State.LUNCH), State.BREAKFAST), State.LUNCH); assertStateChange(stateMachine, () -> assertEquals(stateMachine.set(State.BREAKFAST), State.LUNCH), State.BREAKFAST); // transition to a final state assertStateChange(stateMachine, () -> assertEquals(stateMachine.set(State.DINNER), State.BREAKFAST), State.DINNER); // attempt transition from a final state assertNoStateChange(stateMachine, () -> { try { stateMachine.set(State.LUNCH); fail("expected IllegalStateException"); } catch (IllegalStateException expected) { } }); assertNoStateChange(stateMachine, () -> stateMachine.set(State.DINNER)); }
@Override public boolean isFinished() { return state.get() == FINISHED; }
@Test public void testCompareAndSet() throws Exception { StateMachine<State> stateMachine = new StateMachine<>("test", executor, State.BREAKFAST, ImmutableSet.of(State.DINNER)); assertEquals(stateMachine.get(), State.BREAKFAST); // no match with new state assertNoStateChange(stateMachine, () -> stateMachine.compareAndSet(State.DINNER, State.LUNCH)); // match with new state assertStateChange(stateMachine, () -> stateMachine.compareAndSet(State.BREAKFAST, State.LUNCH), State.LUNCH); // no match with same state assertNoStateChange(stateMachine, () -> stateMachine.compareAndSet(State.BREAKFAST, State.LUNCH)); // match with same state assertNoStateChange(stateMachine, () -> stateMachine.compareAndSet(State.LUNCH, State.LUNCH)); // transition to a final state assertStateChange(stateMachine, () -> stateMachine.compareAndSet(State.LUNCH, State.DINNER), State.DINNER); // attempt transition from a final state assertNoStateChange(stateMachine, () -> { try { stateMachine.compareAndSet(State.DINNER, State.LUNCH); fail("expected IllegalStateException"); } catch (IllegalStateException expected) { } }); assertNoStateChange(stateMachine, () -> stateMachine.compareAndSet(State.DINNER, State.DINNER)); }
throws Exception StateMachine<State> stateMachine = new StateMachine<>("test", executor, State.BREAKFAST, ImmutableSet.of(State.DINNER)); assertEquals(stateMachine.get(), State.BREAKFAST); () -> assertFalse(stateMachine.setIf(State.LUNCH, currentState -> { assertEquals(currentState, State.BREAKFAST); return false; () -> assertTrue(stateMachine.setIf(State.LUNCH, currentState -> { assertEquals(currentState, State.BREAKFAST); return true; () -> assertFalse(stateMachine.setIf(State.LUNCH, currentState -> { assertEquals(currentState, State.LUNCH); return false; () -> assertFalse(stateMachine.setIf(State.LUNCH, currentState -> { assertEquals(currentState, State.LUNCH); return true; assertStateChange(stateMachine, () -> stateMachine.setIf(State.DINNER, currentState -> true), State.DINNER); stateMachine.setIf(State.LUNCH, currentState -> true); fail("expected IllegalStateException"); assertNoStateChange(stateMachine, () -> stateMachine.setIf(State.LUNCH, currentState -> false)); assertNoStateChange(stateMachine, () -> stateMachine.setIf(State.DINNER, currentState -> true));
@Override public void addStateChangeListener(StateChangeListener<BufferState> stateChangeListener) { state.addStateChangeListener(stateChangeListener); }
public LazyOutputBuffer( TaskId taskId, String taskInstanceId, Executor executor, DataSize maxBufferSize, Supplier<LocalMemoryContext> systemMemoryContextSupplier) { requireNonNull(taskId, "taskId is null"); this.taskInstanceId = requireNonNull(taskInstanceId, "taskInstanceId is null"); this.executor = requireNonNull(executor, "executor is null"); state = new StateMachine<>(taskId + "-buffer", executor, OPEN, TERMINAL_BUFFER_STATES); this.maxBufferSize = requireNonNull(maxBufferSize, "maxBufferSize is null"); checkArgument(maxBufferSize.toBytes() > 0, "maxBufferSize must be at least 1"); this.systemMemoryContextSupplier = requireNonNull(systemMemoryContextSupplier, "systemMemoryContextSupplier is null"); }
/** * Sets the state. * If the new state does not {@code .equals()} the current state, listeners and waiters will be notified. * * @return the old state */ public T set(T newState) { checkState(!Thread.holdsLock(lock), "Can not set state while holding the lock"); requireNonNull(newState, "newState is null"); T oldState; FutureStateChange<T> futureStateChange; ImmutableList<StateChangeListener<T>> stateChangeListeners; synchronized (lock) { if (state.equals(newState)) { return state; } checkState(!isTerminalState(state), "%s can not transition from %s to %s", name, state, newState); oldState = state; state = newState; futureStateChange = this.futureStateChange.getAndSet(new FutureStateChange<>()); stateChangeListeners = ImmutableList.copyOf(this.stateChangeListeners); // if we are now in a terminal state, free the listeners since this will be the last notification if (isTerminalState(state)) { this.stateChangeListeners.clear(); } } fireStateChanged(newState, futureStateChange, stateChangeListeners); return oldState; }
public synchronized boolean transitionToScheduling() { return stageState.compareAndSet(PLANNED, SCHEDULING); }
/** * Add a listener for the final task info. This notification is guaranteed to be fired only once. * Listener is always notified asynchronously using a dedicated notification thread pool so, care should * be taken to avoid leaking {@code this} when adding a listener in a constructor. Additionally, it is * possible notifications are observed out of order due to the asynchronous execution. */ public void addFinalTaskInfoListener(StateChangeListener<TaskInfo> stateChangeListener) { AtomicBoolean done = new AtomicBoolean(); StateChangeListener<Optional<TaskInfo>> fireOnceStateChangeListener = finalTaskInfo -> { if (finalTaskInfo.isPresent() && done.compareAndSet(false, true)) { stateChangeListener.stateChanged(finalTaskInfo.get()); } }; finalTaskInfo.addStateChangeListener(fireOnceStateChangeListener); fireOnceStateChangeListener.stateChanged(finalTaskInfo.get()); }
/** * Fail the buffer, discarding all pages, but blocking readers. */ public synchronized void fail() { // ignore fail if the buffer already in a terminal state. if (state.get().isTerminal()) { return; } state.set(FAILED); partitionBuffers.values().forEach(PartitionBuffer::destroy); // DO NOT free readers }
synchronized void updateTaskInfo(TaskInfo newValue) { boolean updated = taskInfo.setIf(newValue, oldValue -> { TaskStatus oldTaskStatus = oldValue.getTaskStatus(); TaskStatus newTaskStatus = newValue.getTaskStatus(); if (oldTaskStatus.getState().isDone()) { // never update if the task has reached a terminal state return false; } // don't update to an older version (same version is ok) return newTaskStatus.getVersion() >= oldTaskStatus.getVersion(); }); if (updated && newValue.getTaskStatus().getState().isDone()) { finalTaskInfo.compareAndSet(Optional.empty(), Optional.of(newValue)); stop(); } }
/** * Adds a listener to be notified when the state instance changes according to {@code .equals()}. * Listener is always notified asynchronously using a dedicated notification thread pool so, care should * be taken to avoid leaking {@code this} when adding a listener in a constructor. Additionally, it is * possible notifications are observed out of order due to the asynchronous execution. The listener is * immediately notified immediately of the current state. */ public void addStateChangeListener(StateChangeListener<T> stateChangeListener) { requireNonNull(stateChangeListener, "stateChangeListener is null"); boolean inTerminalState; T currentState; synchronized (lock) { currentState = state; inTerminalState = isTerminalState(currentState); if (!inTerminalState) { stateChangeListeners.add(stateChangeListener); } } // fire state change listener with the current state // always fire listener callbacks from a different thread safeExecute(() -> stateChangeListener.stateChanged(currentState)); }
/** * Gets a future that completes when the state is no longer {@code .equals()} to {@code currentState)}. */ public ListenableFuture<T> getStateChange(T currentState) { checkState(!Thread.holdsLock(lock), "Can not wait for state change while holding the lock"); requireNonNull(currentState, "currentState is null"); synchronized (lock) { // return a completed future if the state has already changed, or we are in a terminal state if (!state.equals(currentState) || isTerminalState(state)) { return immediateFuture(state); } return futureStateChange.get().createNewListener(); } }