Class RegionRemoteProcedureBase
java.lang.Object
org.apache.hadoop.hbase.procedure2.Procedure<MasterProcedureEnv>
org.apache.hadoop.hbase.master.assignment.RegionRemoteProcedureBase
- All Implemented Interfaces:
Comparable<Procedure<MasterProcedureEnv>>
,TableProcedureInterface
,RemoteProcedureDispatcher.RemoteProcedure<MasterProcedureEnv,
ServerName>
- Direct Known Subclasses:
CloseRegionProcedure
,OpenRegionProcedure
@Private
public abstract class RegionRemoteProcedureBase
extends Procedure<MasterProcedureEnv>
implements TableProcedureInterface, RemoteProcedureDispatcher.RemoteProcedure<MasterProcedureEnv,ServerName>
The base class for the remote procedures used to open/close a region.
Notice that here we do not care about the result of the remote call, if the remote call is
finished, either succeeded or not, we will always finish the procedure. The parent procedure
should take care of the result and try to reschedule if the result is not good.
-
Nested Class Summary
Nested classes/interfaces inherited from class org.apache.hadoop.hbase.procedure2.Procedure
Procedure.LockState
Nested classes/interfaces inherited from interface org.apache.hadoop.hbase.master.procedure.TableProcedureInterface
TableProcedureInterface.TableOperationType
-
Field Summary
Modifier and TypeFieldDescriptionprivate CompletableFuture<Void>
private static final org.slf4j.Logger
protected RegionInfo
private RetryCounter
private long
private org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionRemoteProcedureBaseState
protected ServerName
private org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode
Fields inherited from class org.apache.hadoop.hbase.procedure2.Procedure
NO_PROC_ID, NO_TIMEOUT
Fields inherited from interface org.apache.hadoop.hbase.master.procedure.TableProcedureInterface
DUMMY_NAMESPACE_TABLE_NAME
-
Constructor Summary
ModifierConstructorDescriptionprotected
protected
RegionRemoteProcedureBase
(TransitRegionStateProcedure parent, RegionInfo region, ServerName targetServer) -
Method Summary
Modifier and TypeMethodDescriptionprotected boolean
abort
(MasterProcedureEnv env) The abort() call is asynchronous and each procedure must decide how to deal with it, if they want to be abortable.protected void
Called when the procedure is ready to be added to the queue after the loading/replay operation.protected abstract void
checkTransition
(RegionStateNode regionNode, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode transitionCode, long seqId) protected void
deserializeStateData
(ProcedureStateSerializer serializer) Called on store load to allow the user to decode the previously serialized state.protected Procedure<MasterProcedureEnv>[]
The main code of the procedure.private CompletableFuture<Void>
private TransitRegionStateProcedure
private RegionStateNode
Returns the name of the table the procedure is operating onprotected abstract RemoteProcedureDispatcher.RemoteOperation
private void
persistAndWake
(MasterProcedureEnv env, RegionStateNode regionNode) remoteCallBuild
(MasterProcedureEnv env, ServerName remote) For building the remote operation.void
remoteCallFailed
(MasterProcedureEnv env, ServerName remote, IOException exception) Called when the executeProcedure call is failed.void
Called when RS tells the remote procedure is succeeded through thereportProcedureDone
method.void
Called when RS tells the remote procedure is failed through thereportProcedureDone
method.(package private) void
reportTransition
(MasterProcedureEnv env, RegionStateNode regionNode, ServerName serverName, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode transitionCode, long seqId) protected abstract void
restoreSucceedState
(AssignmentManager am, RegionStateNode regionNode, long seqId) protected void
The code to undo what was done by the execute() code.protected void
serializeStateData
(ProcedureStateSerializer serializer) The user-level code of the procedure may have some state to persist (e.g.(package private) void
serverCrashed
(MasterProcedureEnv env, RegionStateNode regionNode, ServerName serverName) private void
protected boolean
Called by the ProcedureExecutor when the timeout set by setTimeout() is expired.(package private) void
stateLoaded
(AssignmentManager am, RegionStateNode regionNode) boolean
Whether store this remote procedure in dispatched queue only OpenRegionProcedure and CloseRegionProcedure return false since they are not fully controlled by dispatcherprotected void
toStringClassDetails
(StringBuilder builder) Extend the toString() information with the procedure details e.g.private void
protected abstract void
updateTransitionWithoutPersistingToMeta
(MasterProcedureEnv env, RegionStateNode regionNode, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode transitionCode, long seqId) protected boolean
TheProcedure.doAcquireLock(Object, ProcedureStore)
will be split into two steps, first, it will call us to determine whether we need to wait for initialization, second, it will callProcedure.acquireLock(Object)
to actually handle the lock for this procedure.Methods inherited from class org.apache.hadoop.hbase.procedure2.Procedure
acquireLock, addStackIndex, beforeReplay, bypass, compareTo, completionCleanup, doExecute, doRollback, elapsedTime, getChildrenLatch, getException, getLastUpdate, getNonceKey, getOwner, getParentProcId, getProcedureMetrics, getProcId, getProcIdHashCode, getResult, getRootProcedureId, getRootProcId, getStackIndexes, getState, getSubmittedTime, getTimeout, getTimeoutTimestamp, hasChildren, hasException, hasLock, hasOwner, hasParent, hasTimeout, haveSameParent, holdLock, incChildrenLatch, isBypass, isFailed, isFinished, isInitializing, isLockedWhenLoading, isRollbackSupported, isRunnable, isSuccess, isWaiting, isYieldAfterExecutionStep, releaseLock, removeStackIndex, setAbortFailure, setChildrenLatch, setExecuted, setFailure, setFailure, setLastUpdate, setNonceKey, setOwner, setOwner, setParentProcId, setProcId, setResult, setRootProcId, setStackIndexes, setState, setSubmittedTime, setTimeout, shouldWaitClientAck, skipPersistence, suspend, toString, toStringClass, toStringDetails, toStringSimpleSB, toStringState, updateMetricsOnFinish, updateMetricsOnSubmit, updateTimestamp, wasExecuted
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
Methods inherited from interface org.apache.hadoop.hbase.master.procedure.TableProcedureInterface
getTableOperationType
-
Field Details
-
LOG
-
region
-
targetServer
-
state
private org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionRemoteProcedureBaseState state -
transitionCode
private org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode transitionCode -
seqId
-
retryCounter
-
future
-
-
Constructor Details
-
RegionRemoteProcedureBase
protected RegionRemoteProcedureBase() -
RegionRemoteProcedureBase
protected RegionRemoteProcedureBase(TransitRegionStateProcedure parent, RegionInfo region, ServerName targetServer)
-
-
Method Details
-
remoteCallBuild
public Optional<RemoteProcedureDispatcher.RemoteOperation> remoteCallBuild(MasterProcedureEnv env, ServerName remote) Description copied from interface:RemoteProcedureDispatcher.RemoteProcedure
For building the remote operation. May be empty if no need to send remote call. Usually, this means the RemoteProcedure has been finished already. This is possible, as we may have already sent the procedure to RS but then the rpc connection is broken so the executeProcedures call fails, but the RS does receive the procedure and execute it and then report back, before we retry again.- Specified by:
remoteCallBuild
in interfaceRemoteProcedureDispatcher.RemoteProcedure<MasterProcedureEnv,
ServerName>
-
newRemoteOperation
protected abstract RemoteProcedureDispatcher.RemoteOperation newRemoteOperation(MasterProcedureEnv env) -
remoteOperationCompleted
Description copied from interface:RemoteProcedureDispatcher.RemoteProcedure
Called when RS tells the remote procedure is succeeded through thereportProcedureDone
method.- Specified by:
remoteOperationCompleted
in interfaceRemoteProcedureDispatcher.RemoteProcedure<MasterProcedureEnv,
ServerName>
-
remoteOperationFailed
Description copied from interface:RemoteProcedureDispatcher.RemoteProcedure
Called when RS tells the remote procedure is failed through thereportProcedureDone
method.- Specified by:
remoteOperationFailed
in interfaceRemoteProcedureDispatcher.RemoteProcedure<MasterProcedureEnv,
ServerName>
-
getRegionNode
-
remoteCallFailed
Description copied from interface:RemoteProcedureDispatcher.RemoteProcedure
Called when the executeProcedure call is failed.- Specified by:
remoteCallFailed
in interfaceRemoteProcedureDispatcher.RemoteProcedure<MasterProcedureEnv,
ServerName>
-
getTableName
Description copied from interface:TableProcedureInterface
Returns the name of the table the procedure is operating on- Specified by:
getTableName
in interfaceTableProcedureInterface
-
waitInitialized
Description copied from class:Procedure
TheProcedure.doAcquireLock(Object, ProcedureStore)
will be split into two steps, first, it will call us to determine whether we need to wait for initialization, second, it will callProcedure.acquireLock(Object)
to actually handle the lock for this procedure. This is because that when master restarts, we need to restore the lock state for all the procedures to not break the semantic ifProcedure.holdLock(Object)
is true. But theProcedureExecutor
will be started before the master finish initialization(as it is part of the initialization!), so we need to split the code into two steps, and when restore, we just restore the lock part and ignore the waitInitialized part. Otherwise there will be dead lock.- Overrides:
waitInitialized
in classProcedure<MasterProcedureEnv>
- Returns:
- true means we need to wait until the environment has been initialized, otherwise true.
-
rollback
Description copied from class:Procedure
The code to undo what was done by the execute() code. It is called when the procedure or one of the sub-procedures failed or an abort was requested. It should cleanup all the resources created by the execute() call. The implementation must be idempotent since rollback() may be called multiple time in case of machine failure in the middle of the execution.- Specified by:
rollback
in classProcedure<MasterProcedureEnv>
- Parameters:
env
- the environment passed to the ProcedureExecutor- Throws:
IOException
- temporary failure, the rollback will retry laterInterruptedException
- the procedure will be added back to the queue and retried later
-
abort
Description copied from class:Procedure
The abort() call is asynchronous and each procedure must decide how to deal with it, if they want to be abortable. The simplest implementation is to have an AtomicBoolean set in the abort() method and then the execute() will check if the abort flag is set or not. abort() may be called multiple times from the client, so the implementation must be idempotent.NOTE: abort() is not like Thread.interrupt(). It is just a notification that allows the procedure implementor abort.
- Specified by:
abort
in classProcedure<MasterProcedureEnv>
-
checkTransition
protected abstract void checkTransition(RegionStateNode regionNode, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode transitionCode, long seqId) throws UnexpectedStateException - Throws:
UnexpectedStateException
-
updateTransitionWithoutPersistingToMeta
protected abstract void updateTransitionWithoutPersistingToMeta(MasterProcedureEnv env, RegionStateNode regionNode, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode transitionCode, long seqId) throws IOException - Throws:
IOException
-
persistAndWake
-
reportTransition
void reportTransition(MasterProcedureEnv env, RegionStateNode regionNode, ServerName serverName, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode transitionCode, long seqId) throws IOException - Throws:
IOException
-
serverCrashed
-
restoreSucceedState
protected abstract void restoreSucceedState(AssignmentManager am, RegionStateNode regionNode, long seqId) throws IOException - Throws:
IOException
-
stateLoaded
-
getParent
-
unattach
-
getFuture
-
setFuture
-
execute
protected Procedure<MasterProcedureEnv>[] execute(MasterProcedureEnv env) throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException Description copied from class:Procedure
The main code of the procedure. It must be idempotent since execute() may be called multiple times in case of machine failure in the middle of the execution.- Specified by:
execute
in classProcedure<MasterProcedureEnv>
- Parameters:
env
- the environment passed to the ProcedureExecutor- Returns:
- a set of sub-procedures to run or ourselves if there is more work to do or null if the procedure is done.
- Throws:
ProcedureYieldException
- the procedure will be added back to the queue and retried later.ProcedureSuspendedException
- 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.InterruptedException
- the procedure will be added back to the queue and retried later.
-
setTimeoutFailure
Description copied from class:Procedure
Called by the ProcedureExecutor when the timeout set by setTimeout() is expired. Another usage for this method is to implement retrying. A procedure can set the state toWAITING_TIMEOUT
by callingsetState
method, and throw aProcedureSuspendedException
to halt the execution of the procedure, and do not forget a callProcedure.setTimeout(int)
method to set the timeout. And you should also override this method to wake up the procedure, and also return false to tell the ProcedureExecutor that the timeout event has been handled.- Overrides:
setTimeoutFailure
in classProcedure<MasterProcedureEnv>
- Returns:
- true to let the framework handle the timeout as abort, false in case the procedure handled the timeout itself.
-
storeInDispatchedQueue
Description copied from interface:RemoteProcedureDispatcher.RemoteProcedure
Whether store this remote procedure in dispatched queue only OpenRegionProcedure and CloseRegionProcedure return false since they are not fully controlled by dispatcher- Specified by:
storeInDispatchedQueue
in interfaceRemoteProcedureDispatcher.RemoteProcedure<MasterProcedureEnv,
ServerName>
-
serializeStateData
Description copied from class:Procedure
The user-level code of the procedure may have some state to persist (e.g. input arguments or current position in the processing state) to be able to resume on failure.- Specified by:
serializeStateData
in classProcedure<MasterProcedureEnv>
- Parameters:
serializer
- stores the serializable state- Throws:
IOException
-
deserializeStateData
Description copied from class:Procedure
Called on store load to allow the user to decode the previously serialized state.- Specified by:
deserializeStateData
in classProcedure<MasterProcedureEnv>
- Parameters:
serializer
- contains the serialized state- Throws:
IOException
-
afterReplay
Description copied from class:Procedure
Called when the procedure is ready to be added to the queue after the loading/replay operation.- Overrides:
afterReplay
in classProcedure<MasterProcedureEnv>
-
getProcName
- Overrides:
getProcName
in classProcedure<MasterProcedureEnv>
-
toStringClassDetails
Description copied from class:Procedure
Extend the toString() information with the procedure details e.g. className and parameters- Overrides:
toStringClassDetails
in classProcedure<MasterProcedureEnv>
- Parameters:
builder
- the string builder to use to append the proc specific information
-