Class LockProcedure
java.lang.Object
org.apache.hadoop.hbase.procedure2.Procedure<MasterProcedureEnv>
org.apache.hadoop.hbase.master.locking.LockProcedure
- All Implemented Interfaces:
Comparable<Procedure<MasterProcedureEnv>>
,TableProcedureInterface
@Private
public final class LockProcedure
extends Procedure<MasterProcedureEnv>
implements TableProcedureInterface
Procedure to allow blessed clients and external admin tools to take our internal Schema locks
used by the procedure framework isolating procedures doing creates/deletes etc. on
table/namespace/regions. This procedure when scheduled, acquires specified locks, suspends itself
and waits for:
- Call to unlock: if lock request came from the process itself, say master chore.
- Timeout : if lock request came from RPC. On timeout, evaluates if it should continue holding the lock or not based on last heartbeat timestamp.
-
Nested Class Summary
Modifier and TypeClassDescriptionprivate static interface
private class
private class
private class
private class
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 TypeFieldDescriptionstatic final int
static final int
private String
private final ProcedureEvent<LockProcedure>
private final AtomicLong
static final String
private LockProcedure.LockInterface
private final CountDownLatch
private final AtomicBoolean
private static final org.slf4j.Logger
private String
private boolean
private RegionInfo[]
static final String
private boolean
private TableName
private LockType
private final AtomicBoolean
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
ModifierConstructorDescriptionLockProcedure
(org.apache.hadoop.conf.Configuration conf, String namespace, LockType type, String description, CountDownLatch lockAcquireLatch) Constructor for namespace lock.LockProcedure
(org.apache.hadoop.conf.Configuration conf, RegionInfo[] regionInfos, LockType type, String description, CountDownLatch lockAcquireLatch) Constructor for region lock(s).private
LockProcedure
(org.apache.hadoop.conf.Configuration conf, LockType type, String description, CountDownLatch lockAcquireLatch) LockProcedure
(org.apache.hadoop.conf.Configuration conf, TableName tableName, LockType type, String description, CountDownLatch lockAcquireLatch) Constructor for table lock. -
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 Procedure.LockState
The user should override this method if they need a lock on an Entity.protected void
On recovery, re-execute from start to acquire the locks.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.Returns the name of the table the procedure is operating onGiven an operation type we can take decisions about what to do with pending operations.getType()
private boolean
boolean
Used to keep the procedure lock even when the procedure is yielding or suspended.boolean
isLocked()
protected void
The user should override this method, and release lock if necessary.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.protected boolean
Re run the procedure after every timeout to write new WAL entries so we don't hold back old WALs.private LockProcedure.LockInterface
private LockProcedure.LockInterface
private LockProcedure.LockInterface
private LockProcedure.LockInterface
protected void
toStringClassDetails
(StringBuilder builder) Extend the toString() information with the procedure details e.g.void
unlock
(MasterProcedureEnv env) void
Updates timeout deadline for the lock.Methods inherited from class org.apache.hadoop.hbase.procedure2.Procedure
addStackIndex, afterReplay, bypass, compareTo, completionCleanup, doExecute, 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, incChildrenLatch, isBypass, isFailed, isFinished, isInitializing, isLockedWhenLoading, isRollbackSupported, isRunnable, isSuccess, isWaiting, isYieldAfterExecutionStep, 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, waitInitialized, wasExecuted
-
Field Details
-
LOG
-
DEFAULT_REMOTE_LOCKS_TIMEOUT_MS
- See Also:
-
REMOTE_LOCKS_TIMEOUT_MS_CONF
- See Also:
-
DEFAULT_LOCAL_MASTER_LOCKS_TIMEOUT_MS
- See Also:
-
LOCAL_MASTER_LOCKS_TIMEOUT_MS_CONF
- See Also:
-
namespace
-
tableName
-
regionInfos
-
type
-
lock
-
opType
-
description
-
recoveredMasterLock
-
event
-
locked
-
lastHeartBeat
-
unlock
-
lockAcquireLatch
-
suspended
-
-
Constructor Details
-
LockProcedure
public LockProcedure() -
LockProcedure
private LockProcedure(org.apache.hadoop.conf.Configuration conf, LockType type, String description, CountDownLatch lockAcquireLatch) -
LockProcedure
public LockProcedure(org.apache.hadoop.conf.Configuration conf, String namespace, LockType type, String description, CountDownLatch lockAcquireLatch) throws IllegalArgumentException Constructor for namespace lock.- Parameters:
lockAcquireLatch
- if not null, the latch is decreased when lock is acquired.- Throws:
IllegalArgumentException
-
LockProcedure
public LockProcedure(org.apache.hadoop.conf.Configuration conf, TableName tableName, LockType type, String description, CountDownLatch lockAcquireLatch) throws IllegalArgumentException Constructor for table lock.- Parameters:
lockAcquireLatch
- if not null, the latch is decreased when lock is acquired.- Throws:
IllegalArgumentException
-
LockProcedure
public LockProcedure(org.apache.hadoop.conf.Configuration conf, RegionInfo[] regionInfos, LockType type, String description, CountDownLatch lockAcquireLatch) throws IllegalArgumentException Constructor for region lock(s).- Parameters:
lockAcquireLatch
- if not null, the latch is decreased when lock is acquired. Useful for locks acquired locally from master process.- Throws:
IllegalArgumentException
- if all regions are not from same table.
-
-
Method Details
-
getTableName
Description copied from interface:TableProcedureInterface
Returns the name of the table the procedure is operating on- Specified by:
getTableName
in interfaceTableProcedureInterface
-
getTableOperationType
Description copied from interface:TableProcedureInterface
Given an operation type we can take decisions about what to do with pending operations. e.g. if we get a delete and we have some table operation pending (e.g. add column) we can abort those operations.- Specified by:
getTableOperationType
in interfaceTableProcedureInterface
- Returns:
- the operation type that the procedure is executing.
-
hasHeartbeatExpired
-
updateHeartBeat
Updates timeout deadline for the lock. -
setTimeoutFailure
Re run the procedure after every timeout to write new WAL entries so we don't hold back old WALs.- Overrides:
setTimeoutFailure
in classProcedure<MasterProcedureEnv>
- Returns:
- false, so procedure framework doesn't mark this procedure as failure.
-
unlock
-
execute
protected Procedure<MasterProcedureEnv>[] execute(MasterProcedureEnv env) throws ProcedureSuspendedException 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:
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.
-
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
-
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>
-
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
-
acquireLock
Description copied from class:Procedure
The user should override this method if they need a lock on an Entity. A lock can be anything, and it is up to the implementor. The Procedure Framework will call this method just before it invokesProcedure.execute(Object)
. It callsProcedure.releaseLock(Object)
after the call to execute. If you need to hold the lock for the life of the Procedure -- i.e. you do not want any other Procedure interfering while this Procedure is running, seeProcedure.holdLock(Object)
. Example: in our Master we can execute request in parallel for different tables. We can create t1 and create t2 and these creates can be executed at the same time. Anything else on t1/t2 is queued waiting that specific table create to happen. There are 3 LockState:- LOCK_ACQUIRED should be returned when the proc has the lock and the proc is ready to execute.
- LOCK_YIELD_WAIT should be returned when the proc has not the lock and the framework should take care of readding the procedure back to the runnable set for retry
- LOCK_EVENT_WAIT should be returned when the proc has not the lock and someone will take care of readding the procedure back to the runnable set when the lock is available.
- Overrides:
acquireLock
in classProcedure<MasterProcedureEnv>
- Returns:
- the lock state as described above.
-
releaseLock
Description copied from class:Procedure
The user should override this method, and release lock if necessary.- Overrides:
releaseLock
in classProcedure<MasterProcedureEnv>
-
beforeReplay
On recovery, re-execute from start to acquire the locks. Need to explicitly set it to RUNNABLE because the procedure might have been in WAITING_TIMEOUT state when crash happened. In which case, it'll be sent back to timeout queue on recovery, which we don't want since we want to require locks.- Overrides:
beforeReplay
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
-
getType
-
setupLock
- Throws:
IllegalArgumentException
-
setupNamespaceLock
- Throws:
IllegalArgumentException
-
setupTableLock
- Throws:
IllegalArgumentException
-
setupRegionLock
- Throws:
IllegalArgumentException
-
getDescription
-
isLocked
-
holdLock
Description copied from class:Procedure
Used to keep the procedure lock even when the procedure is yielding or suspended.- Overrides:
holdLock
in classProcedure<MasterProcedureEnv>
- Returns:
- true if the procedure should hold on the lock until completionCleanup()
-