@InterfaceAudience.Private @InterfaceStability.Evolving public abstract class StateMachineProcedure<TEnvironment,TState> extends Procedure<TEnvironment>
The procedure implementor must have an enum of 'states', describing the various step of the procedure. Once the procedure is running, the procedure-framework will call executeFromState() using the 'state' provided by the user. The first call to executeFromState() will be performed with 'state = null'. The implementor can jump between states using setNextState(MyStateEnum.ordinal()). The rollback will call rollbackState() for each state that was executed, in reverse order.
| Modifier and Type | Class and Description |
|---|---|
static class |
StateMachineProcedure.Flow |
Procedure.LockState| Modifier and Type | Field and Description |
|---|---|
private AtomicBoolean |
aborted |
private int |
cycles
Cycles on same state.
|
private static int |
EOF_STATE |
private static org.slf4j.Logger |
LOG |
private int |
previousState
Ordinal of the previous state.
|
private int |
stateCount |
private StateMachineProcedure.Flow |
stateFlow |
private int[] |
states |
private List<Procedure<TEnvironment>> |
subProcList |
NO_PROC_ID, NO_TIMEOUT| Constructor and Description |
|---|
StateMachineProcedure() |
| Modifier and Type | Method and Description |
|---|---|
protected boolean |
abort(TEnvironment env)
The abort() call is asynchronous and each procedure must decide how to deal
with it, if they want to be abortable.
|
protected <T extends Procedure<TEnvironment>> |
addChildProcedure(T... subProcedure)
Add a child procedure to execute
|
protected void |
deserializeStateData(ProcedureStateSerializer serializer)
Called on store load to allow the user to decode the previously serialized
state.
|
protected Procedure[] |
execute(TEnvironment env)
The main code of the procedure.
|
protected abstract StateMachineProcedure.Flow |
executeFromState(TEnvironment env,
TState state)
called to perform a single step of the specified 'state' of the procedure
|
protected void |
failIfAborted()
If procedure has more states then abort it otherwise procedure is finished and abort can be
ignored.
|
protected TState |
getCurrentState() |
int |
getCurrentStateId()
This method is used from test code as it cannot be assumed that state transition will happen
sequentially.
|
protected int |
getCycles() |
protected abstract TState |
getInitialState()
Return the initial state object that will be used for the first call to executeFromState().
|
protected abstract TState |
getState(int stateId)
Convert an ordinal (or state id) to an Enum (or more descriptive) state object.
|
protected abstract int |
getStateId(TState state)
Convert the Enum (or more descriptive) state object to an ordinal (or state id).
|
private boolean |
hasMoreState() |
private boolean |
isEofState() |
protected boolean |
isRollbackSupported(TState state)
Used by the default implementation of abort() to know if the current state can be aborted
and rollback can be triggered.
|
protected boolean |
isYieldAfterExecutionStep(TEnvironment env)
By default, the procedure framework/executor will try to run procedures start to finish.
|
protected boolean |
isYieldBeforeExecuteFromState(TEnvironment env,
TState state)
By default, the executor will try ro run all the steps of the procedure start to finish.
|
protected void |
rollback(TEnvironment env)
The code to undo what was done by the execute() code.
|
protected abstract void |
rollbackState(TEnvironment env,
TState state)
called to perform the rollback of the specified state
|
protected void |
serializeStateData(ProcedureStateSerializer serializer)
The user-level code of the procedure may have some state to
persist (e.g.
|
private void |
setNextState(int stateId)
Set the next state for the procedure.
|
protected void |
setNextState(TState state)
Set the next state for the procedure.
|
protected void |
toStringState(StringBuilder builder)
Called from
Procedure.toString() when interpolating Procedure State. |
acquireLock, addStackIndex, afterReplay, beforeReplay, bypass, compareTo, completionCleanup, doAcquireLock, doExecute, doReleaseLock, doRollback, elapsedTime, getChildrenLatch, getException, getLastUpdate, getNonceKey, getOwner, getParentProcId, getProcedureMetrics, getProcId, getProcIdHashCode, getProcName, getResult, getRootProcedureId, getRootProcId, getStackIndexes, getState, getSubmittedTime, getTimeout, getTimeoutTimestamp, hasChildren, hasException, hasLock, hasOwner, hasParent, hasTimeout, haveSameParent, holdLock, incChildrenLatch, isBypass, isFailed, isFinished, isInitializing, isLockedWhenLoading, isRunnable, isSuccess, isWaiting, lockedWhenLoading, needPersistence, releaseLock, removeStackIndex, resetPersistence, restoreLock, setAbortFailure, setChildrenLatch, setFailure, setFailure, setLastUpdate, setNonceKey, setOwner, setOwner, setParentProcId, setProcId, setResult, setRootProcId, setStackIndexes, setState, setSubmittedTime, setTimeout, setTimeoutFailure, shouldWaitClientAck, skipPersistence, toString, toStringClass, toStringClassDetails, toStringDetails, toStringSimpleSB, tryRunnable, updateMetricsOnFinish, updateMetricsOnSubmit, updateTimestamp, waitInitialized, wasExecutedprivate static final org.slf4j.Logger LOG
private static final int EOF_STATE
private final AtomicBoolean aborted
private StateMachineProcedure.Flow stateFlow
private int stateCount
private int[] states
private List<Procedure<TEnvironment>> subProcList
private int cycles
private int previousState
public StateMachineProcedure()
protected final int getCycles()
protected abstract StateMachineProcedure.Flow executeFromState(TEnvironment env, TState state) throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException
state - state to executeProcedureSuspendedExceptionProcedureYieldExceptionInterruptedExceptionprotected abstract void rollbackState(TEnvironment env, TState state) throws IOException, InterruptedException
state - state to rollbackIOException - temporary failure, the rollback will retry laterInterruptedExceptionprotected abstract TState getState(int stateId)
stateId - the ordinal() of the state enum (or state id)protected abstract int getStateId(TState state)
state - the state enum objectprotected abstract TState getInitialState()
protected void setNextState(TState state)
state - the state enum objectprotected boolean isYieldBeforeExecuteFromState(TEnvironment env, TState state)
state - the state we are going to execute next.protected <T extends Procedure<TEnvironment>> void addChildProcedure(T... subProcedure)
subProcedure - the child procedureprotected Procedure[] execute(TEnvironment env) throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException
Procedureexecute in class Procedure<TEnvironment>env - the environment passed to the ProcedureExecutorProcedureSuspendedException - Signal to the executor that Procedure has suspended itself
and has set itself up waiting for an external event to wake it back up again.ProcedureYieldException - the procedure will be added back to the queue and retried
later.InterruptedException - the procedure will be added back to the queue and retried later.protected void rollback(TEnvironment env) throws IOException, InterruptedException
Procedurerollback in class Procedure<TEnvironment>env - the environment passed to the ProcedureExecutorIOException - temporary failure, the rollback will retry laterInterruptedException - the procedure will be added back to the queue and retried laterprivate boolean isEofState()
protected boolean abort(TEnvironment env)
ProcedureNOTE: abort() is not like Thread.interrupt(). It is just a notification that allows the procedure implementor abort.
abort in class Procedure<TEnvironment>protected final void failIfAborted()
protected boolean isRollbackSupported(TState state)
protected boolean isYieldAfterExecutionStep(TEnvironment env)
ProcedureisYieldAfterExecutionStep in class Procedure<TEnvironment>env - the environment passed to the ProcedureExecutorprivate boolean hasMoreState()
protected TState getCurrentState()
public int getCurrentStateId()
private void setNextState(int stateId)
stateId - the ordinal() of the state enum (or state id)protected void toStringState(StringBuilder builder)
ProcedureProcedure.toString() when interpolating Procedure State. Allows decorating
generic Procedure State with Procedure particulars.toStringState in class Procedure<TEnvironment>builder - Append current ProcedureProtos.ProcedureStateprotected void serializeStateData(ProcedureStateSerializer serializer) throws IOException
ProcedureserializeStateData in class Procedure<TEnvironment>serializer - stores the serializable stateIOExceptionprotected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException
ProceduredeserializeStateData in class Procedure<TEnvironment>serializer - contains the serialized stateIOExceptionCopyright © 2007–2020 The Apache Software Foundation. All rights reserved.