@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 | cyclesCycles on same state. | 
| private static int | EOF_STATE | 
| private static org.slf4j.Logger | LOG | 
| private int | previousStateOrdinal of the previous state. | 
| protected 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() | 
| protected 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 interpolatingProcedureState. | 
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
protected 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 laterprotected 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–2021 The Apache Software Foundation. All rights reserved.