Class MergeTableRegionsProcedure
java.lang.Object
org.apache.hadoop.hbase.procedure2.Procedure<TEnvironment>
org.apache.hadoop.hbase.procedure2.StateMachineProcedure<MasterProcedureEnv,TState>
org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState>
org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure
- All Implemented Interfaces:
Comparable<Procedure<MasterProcedureEnv>>
,TableProcedureInterface
@Private
public class MergeTableRegionsProcedure
extends AbstractStateMachineTableProcedure<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState>
The procedure to Merge regions in a table. This procedure takes an exclusive table lock since it
is working over multiple regions. It holds the lock for the life of the procedure. Throws
exception on construction if determines context hostile to merge (cluster going down or master is
shutting down or table is disabled).
-
Nested Class Summary
Nested classes/interfaces inherited from class org.apache.hadoop.hbase.procedure2.StateMachineProcedure
StateMachineProcedure.Flow
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 boolean
private static final org.slf4j.Logger
private RegionInfo
The resulting merged region.private ServerName
private RegionInfo[]
Two or more regions to merge, the 'merge parents'.Fields inherited from class org.apache.hadoop.hbase.procedure2.StateMachineProcedure
stateCount
Fields inherited from class org.apache.hadoop.hbase.procedure2.Procedure
NO_PROC_ID, NO_TIMEOUT
-
Constructor Summary
ConstructorDescriptionMergeTableRegionsProcedure
(MasterProcedureEnv env, RegionInfo[] regionsToMerge, boolean force) -
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.private void
private static void
checkRegionsToMerge
(MasterProcedureEnv env, RegionInfo[] regions, boolean force) private void
Clean up a merged region on rollback after failure.private TransitRegionStateProcedure[]
private void
Create merged region.private static RegionInfo
createMergedRegionInfo
(RegionInfo[] regionsToMerge) Create merged region info by looking at passed inregionsToMerge
to figure what extremes for start and end keys to use; merged region needs to have an extent sufficient to cover all regions-to-merge.private TransitRegionStateProcedure[]
protected void
deserializeStateData
(ProcedureStateSerializer serializer) Called on store load to allow the user to decode the previously serialized state.protected StateMachineProcedure.Flow
executeFromState
(MasterProcedureEnv env, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState state) called to perform a single step of the specified 'state' of the procedureprotected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState
Return the initial state object that will be used for the first call to executeFromState().(package private) RegionInfo
Returns The merged region.protected ProcedureMetrics
Override this method to provide procedure specific counters for submitted count, failed count and time histogram.private int
private ServerName
The procedure could be restarted from a different machine.protected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState
getState
(int stateId) Convert an ordinal (or state id) to an Enum (or more descriptive) state object.protected int
getStateId
(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState state) Convert the Enum (or more descriptive) state object to an ordinal (or state id).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.protected boolean
Used to keep the procedure lock even when the procedure is yielding or suspended.private boolean
isMergeable
(MasterProcedureEnv env, RegionState rs) protected boolean
isRollbackSupported
(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState state) Used by the default implementation of abort() to know if the current state can be aborted and rollback can be triggered.private List<org.apache.hadoop.fs.Path>
mergeStoreFiles
(MasterProcedureEnv env, HRegionFileSystem regionFs, HRegionFileSystem mergeRegionFs, RegionInfo mergedRegion) private void
Post merge region actionprivate void
Post merge region actionprivate void
Action after rollback a merge table regions action.private void
Pre merge region actionprivate void
Post merge region actionprivate boolean
Prepare merge and do some checkprotected void
The user should override this method, and release lock if necessary.private void
private void
Rollback close regionsprivate void
Action for rollback a merge table after prepare mergeprotected void
rollbackState
(MasterProcedureEnv env, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState state) To rollbackMergeTableRegionsProcedure
, two AssignProcedures are asynchronously submitted for each region to be merged (rollback doesn't wait on the completion of the AssignProcedures) .protected void
serializeStateData
(ProcedureStateSerializer serializer) The user-level code of the procedure may have some state to persist (e.g.private void
Set the region states to MERGING statevoid
Extend the toString() information with the procedure details e.g.private void
Add merged region to META and delete original regions.private void
Methods inherited from class org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure
checkOnline, checkTableModifiable, getUser, getWALRegionDir, isTableEnabled, preflightChecks, releaseSyncLatch, setUser, waitInitialized
Methods inherited from class org.apache.hadoop.hbase.procedure2.StateMachineProcedure
addChildProcedure, execute, failIfAborted, getCurrentState, getCurrentStateId, getCycles, isEofState, isRollbackSupported, isYieldAfterExecutionStep, isYieldBeforeExecuteFromState, rollback, setNextState, toStringState
Methods inherited from class org.apache.hadoop.hbase.procedure2.Procedure
addStackIndex, afterReplay, beforeReplay, bypass, compareTo, completionCleanup, doExecute, doRollback, elapsedTime, getChildrenLatch, getException, getLastUpdate, getNonceKey, getOwner, getParentProcId, getProcId, getProcIdHashCode, getProcName, getResult, getRootProcedureId, getRootProcId, getStackIndexes, getState, getSubmittedTime, getTimeout, getTimeoutTimestamp, hasChildren, hasException, hasLock, hasOwner, hasParent, hasTimeout, haveSameParent, incChildrenLatch, isBypass, isFailed, isFinished, isInitializing, isLockedWhenLoading, isRunnable, isSuccess, isWaiting, removeStackIndex, setAbortFailure, setChildrenLatch, setExecuted, setFailure, setFailure, setLastUpdate, setNonceKey, setOwner, setOwner, setParentProcId, setProcId, setResult, setRootProcId, setStackIndexes, setState, setSubmittedTime, setTimeout, setTimeoutFailure, shouldWaitClientAck, skipPersistence, suspend, toString, toStringClass, toStringDetails, toStringSimpleSB, updateMetricsOnFinish, updateMetricsOnSubmit, updateTimestamp, wasExecuted
-
Field Details
-
LOG
-
regionLocation
-
regionsToMerge
Two or more regions to merge, the 'merge parents'. -
mergedRegion
The resulting merged region. -
force
-
-
Constructor Details
-
MergeTableRegionsProcedure
public MergeTableRegionsProcedure() -
MergeTableRegionsProcedure
public MergeTableRegionsProcedure(MasterProcedureEnv env, RegionInfo[] regionsToMerge, boolean force) throws IOException - Throws:
IOException
-
-
Method Details
-
checkRegionsToMerge
private static void checkRegionsToMerge(MasterProcedureEnv env, RegionInfo[] regions, boolean force) throws MergeRegionException - Throws:
MergeRegionException
- If unable to merge regions for whatever reasons.
-
createMergedRegionInfo
Create merged region info by looking at passed inregionsToMerge
to figure what extremes for start and end keys to use; merged region needs to have an extent sufficient to cover all regions-to-merge. -
executeFromState
protected StateMachineProcedure.Flow executeFromState(MasterProcedureEnv env, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState state) Description copied from class:StateMachineProcedure
called to perform a single step of the specified 'state' of the procedure- Specified by:
executeFromState
in classStateMachineProcedure<MasterProcedureEnv,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState> state
- state to execute- Returns:
- Flow.NO_MORE_STATE if the procedure is completed, Flow.HAS_MORE_STATE if there is another step.
-
rollbackState
protected void rollbackState(MasterProcedureEnv env, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState state) throws IOException To rollbackMergeTableRegionsProcedure
, two AssignProcedures are asynchronously submitted for each region to be merged (rollback doesn't wait on the completion of the AssignProcedures) . This can be improved by changing rollback() to support sub-procedures. See HBASE-19851 for details.- Specified by:
rollbackState
in classStateMachineProcedure<MasterProcedureEnv,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState> state
- state to rollback- Throws:
IOException
- temporary failure, the rollback will retry later
-
isRollbackSupported
protected boolean isRollbackSupported(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState state) Description copied from class:StateMachineProcedure
Used by the default implementation of abort() to know if the current state can be aborted and rollback can be triggered.- Overrides:
isRollbackSupported
in classStateMachineProcedure<MasterProcedureEnv,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState>
-
removeNonDefaultReplicas
- Throws:
IOException
-
checkClosedRegions
- Throws:
IOException
-
getState
protected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState getState(int stateId) Description copied from class:StateMachineProcedure
Convert an ordinal (or state id) to an Enum (or more descriptive) state object.- Specified by:
getState
in classStateMachineProcedure<MasterProcedureEnv,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState> - Parameters:
stateId
- the ordinal() of the state enum (or state id)- Returns:
- the state enum object
-
getStateId
protected int getStateId(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState state) Description copied from class:StateMachineProcedure
Convert the Enum (or more descriptive) state object to an ordinal (or state id).- Specified by:
getStateId
in classStateMachineProcedure<MasterProcedureEnv,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState> - Parameters:
state
- the state enum object- Returns:
- stateId the ordinal() of the state enum (or state id)
-
getInitialState
protected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState getInitialState()Description copied from class:StateMachineProcedure
Return the initial state object that will be used for the first call to executeFromState().- Specified by:
getInitialState
in classStateMachineProcedure<MasterProcedureEnv,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState> - Returns:
- the initial state enum object
-
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.- Overrides:
serializeStateData
in classStateMachineProcedure<MasterProcedureEnv,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState> - 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.- Overrides:
deserializeStateData
in classStateMachineProcedure<MasterProcedureEnv,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState> - Parameters:
serializer
- contains the serialized state- Throws:
IOException
-
toStringClassDetails
Description copied from class:Procedure
Extend the toString() information with the procedure details e.g. className and parameters- Overrides:
toStringClassDetails
in classAbstractStateMachineTableProcedure<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState>
- Parameters:
sb
- the string builder to use to append the proc specific information
-
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 classAbstractStateMachineTableProcedure<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState>
- 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 classAbstractStateMachineTableProcedure<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState>
-
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()
-
getTableName
Description copied from interface:TableProcedureInterface
Returns the name of the table the procedure is operating on- Specified by:
getTableName
in interfaceTableProcedureInterface
- Specified by:
getTableName
in classAbstractStateMachineTableProcedure<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState>
-
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
- Specified by:
getTableOperationType
in classAbstractStateMachineTableProcedure<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState>
- Returns:
- the operation type that the procedure is executing.
-
getProcedureMetrics
Description copied from class:Procedure
Override this method to provide procedure specific counters for submitted count, failed count and time histogram.- Overrides:
getProcedureMetrics
in classProcedure<MasterProcedureEnv>
- Parameters:
env
- The environment passed to the procedure executor- Returns:
- Container object for procedure related metric
-
prepareMergeRegion
Prepare merge and do some check- Throws:
IOException
-
isMergeable
- Throws:
IOException
-
rollbackPrepareMerge
Action for rollback a merge table after prepare merge- Throws:
IOException
-
preMergeRegions
Pre merge region action- Parameters:
env
- MasterProcedureEnv- Throws:
IOException
-
postRollBackMergeRegions
Action after rollback a merge table regions action.- Throws:
IOException
-
setRegionStateToMerging
Set the region states to MERGING state -
createMergedRegion
Create merged region. The way the merge works is that we make a 'merges' temporary directory in the FIRST parent region to merge (Do not change this without also changing the rollback where we look in this FIRST region for the merge dir). We then collect here references to all the store files in all the parent regions including those of the FIRST parent region into a subdirectory, named for the resultant merged region. We then call commitMergeRegion. It finds this subdirectory of storefile references and moves them under the new merge region (creating the region layout as side effect). After assign of the new merge region, we will run a compaction. This will undo the references but the reference files remain in place until the archiver runs (which it does on a period as a chore in the RegionServer that hosts the merge region -- see CompactedHFilesDischarger). Once the archiver has moved aside the no-longer used references, the merge region no longer has references. The catalog janitor will notice when it runs next and it will remove the old parent regions.- Throws:
IOException
-
mergeStoreFiles
private List<org.apache.hadoop.fs.Path> mergeStoreFiles(MasterProcedureEnv env, HRegionFileSystem regionFs, HRegionFileSystem mergeRegionFs, RegionInfo mergedRegion) throws IOException - Throws:
IOException
-
cleanupMergedRegion
Clean up a merged region on rollback after failure.- Throws:
IOException
-
rollbackCloseRegionsForMerge
Rollback close regions- Throws:
IOException
-
createUnassignProcedures
private TransitRegionStateProcedure[] createUnassignProcedures(MasterProcedureEnv env) throws IOException - Throws:
IOException
-
createAssignProcedures
private TransitRegionStateProcedure[] createAssignProcedures(MasterProcedureEnv env) throws IOException - Throws:
IOException
-
getRegionReplication
- Throws:
IOException
-
preMergeRegionsCommit
Post merge region action- Parameters:
env
- MasterProcedureEnv- Throws:
IOException
-
updateMetaForMergedRegions
Add merged region to META and delete original regions.- Throws:
IOException
-
postMergeRegionsCommit
Post merge region action- Parameters:
env
- MasterProcedureEnv- Throws:
IOException
-
postCompletedMergeRegions
Post merge region action- Parameters:
env
- MasterProcedureEnv- Throws:
IOException
-
getServerName
The procedure could be restarted from a different machine. If the variable is null, we need to retrieve it.- Parameters:
env
- MasterProcedureEnv
-
writeMaxSequenceIdFile
- Throws:
IOException
-
getMergedRegion
Returns The merged region. Maybe be null if called to early or we failed. -
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.
- Overrides:
abort
in classStateMachineProcedure<MasterProcedureEnv,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState>
-