@InterfaceAudience.Private public class CreateTableProcedure extends AbstractStateMachineTableProcedure<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>
| Modifier and Type | Class and Description |
|---|---|
protected static interface |
CreateTableProcedure.CreateHdfsRegions |
StateMachineProcedure.FlowProcedure.LockStateTableProcedureInterface.TableOperationType| Modifier and Type | Field and Description |
|---|---|
private static org.slf4j.Logger |
LOG |
private List<RegionInfo> |
newRegions |
private TableDescriptor |
tableDescriptor |
stateCountNO_PROC_ID, NO_TIMEOUT| Constructor and Description |
|---|
CreateTableProcedure() |
CreateTableProcedure(MasterProcedureEnv env,
TableDescriptor tableDescriptor,
RegionInfo[] newRegions) |
CreateTableProcedure(MasterProcedureEnv env,
TableDescriptor tableDescriptor,
RegionInfo[] newRegions,
ProcedurePrepareLatch syncLatch) |
| Modifier and Type | Method and Description |
|---|---|
protected Procedure.LockState |
acquireLock(MasterProcedureEnv env)
The user should override this method if they need a lock on an Entity.
|
private static void |
addRegionsToMeta(MasterProcedureEnv env,
TableDescriptor tableDescriptor,
List<RegionInfo> regionInfos)
Add the specified set of regions to the hbase:meta table.
|
protected static List<RegionInfo> |
addTableToMeta(MasterProcedureEnv env,
TableDescriptor tableDescriptor,
List<RegionInfo> regions) |
protected static List<RegionInfo> |
createFsLayout(MasterProcedureEnv env,
TableDescriptor tableDescriptor,
List<RegionInfo> newRegions) |
protected static List<RegionInfo> |
createFsLayout(MasterProcedureEnv env,
TableDescriptor tableDescriptor,
List<RegionInfo> newRegions,
CreateTableProcedure.CreateHdfsRegions hdfsRegionHandler) |
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.CreateTableState state)
called to perform a single step of the specified 'state' of the procedure
|
(package private) RegionInfo |
getFirstRegionInfo() |
protected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState |
getInitialState()
Return the initial state object that will be used for the first call to executeFromState().
|
protected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState |
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.CreateTableState state)
Convert the Enum (or more descriptive) state object to an ordinal (or state id).
|
TableName |
getTableName() |
TableProcedureInterface.TableOperationType |
getTableOperationType()
Given an operation type we can take decisions about what to do with pending operations.
|
protected boolean |
isRollbackSupported(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState state)
Used by the default implementation of abort() to know if the current state can be aborted
and rollback can be triggered.
|
protected static void |
moveTempDirectoryToHBaseRoot(MasterProcedureEnv env,
TableDescriptor tableDescriptor,
org.apache.hadoop.fs.Path tempTableDir) |
private void |
postCreate(MasterProcedureEnv env) |
private void |
preCreate(MasterProcedureEnv env) |
private boolean |
prepareCreate(MasterProcedureEnv env) |
protected void |
rollbackState(MasterProcedureEnv env,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState 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.
|
protected static void |
setEnabledState(MasterProcedureEnv env,
TableName tableName) |
protected static void |
setEnablingState(MasterProcedureEnv env,
TableName tableName) |
protected boolean |
shouldWaitClientAck(MasterProcedureEnv env)
By default, the executor will keep the procedure result around util
the eviction TTL is expired.
|
protected static void |
updateTableDescCache(MasterProcedureEnv env,
TableName tableName) |
protected boolean |
waitInitialized(MasterProcedureEnv env)
The
Procedure.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 call
Procedure.acquireLock(Object) to actually handle the lock for this procedure. |
checkOnline, checkTableModifiable, getUser, getWALRegionDir, preflightChecks, releaseLock, releaseSyncLatch, setUser, toStringClassDetailsabort, addChildProcedure, execute, failIfAborted, getCurrentState, getCurrentStateId, getCycles, isEofState, isYieldAfterExecutionStep, isYieldBeforeExecuteFromState, rollback, setNextState, toStringStateaddStackIndex, afterReplay, beforeReplay, 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, holdLock, incChildrenLatch, isBypass, isFailed, isFinished, isInitializing, isLockedWhenLoading, isRunnable, isSuccess, isWaiting, removeStackIndex, setAbortFailure, setChildrenLatch, setFailure, setFailure, setLastUpdate, setNonceKey, setOwner, setOwner, setParentProcId, setProcId, setResult, setRootProcId, setStackIndexes, setState, setSubmittedTime, setTimeout, setTimeoutFailure, skipPersistence, toString, toStringClass, toStringDetails, toStringSimpleSB, updateMetricsOnFinish, updateMetricsOnSubmit, updateTimestamp, wasExecutedprivate static final org.slf4j.Logger LOG
private TableDescriptor tableDescriptor
private List<RegionInfo> newRegions
public CreateTableProcedure()
public CreateTableProcedure(MasterProcedureEnv env, TableDescriptor tableDescriptor, RegionInfo[] newRegions)
public CreateTableProcedure(MasterProcedureEnv env, TableDescriptor tableDescriptor, RegionInfo[] newRegions, ProcedurePrepareLatch syncLatch)
protected StateMachineProcedure.Flow executeFromState(MasterProcedureEnv env, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState state) throws InterruptedException
StateMachineProcedureexecuteFromState in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>state - state to executeInterruptedExceptionprotected void rollbackState(MasterProcedureEnv env, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState state) throws IOException
StateMachineProcedurerollbackState in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>state - state to rollbackIOException - temporary failure, the rollback will retry laterprotected boolean isRollbackSupported(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState state)
StateMachineProcedureisRollbackSupported in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>protected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState getState(int stateId)
StateMachineProceduregetState in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>stateId - the ordinal() of the state enum (or state id)protected int getStateId(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState state)
StateMachineProceduregetStateId in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>state - the state enum objectprotected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState getInitialState()
StateMachineProceduregetInitialState in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>public TableName getTableName()
getTableName in interface TableProcedureInterfacegetTableName in class AbstractStateMachineTableProcedure<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>public TableProcedureInterface.TableOperationType getTableOperationType()
TableProcedureInterfacegetTableOperationType in interface TableProcedureInterfacegetTableOperationType in class AbstractStateMachineTableProcedure<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException
ProcedureserializeStateData in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>serializer - stores the serializable stateIOExceptionprotected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException
ProceduredeserializeStateData in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>serializer - contains the serialized stateIOExceptionprotected boolean waitInitialized(MasterProcedureEnv env)
ProcedureProcedure.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 call
Procedure.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 if Procedure.holdLock(Object) is true. But the
ProcedureExecutor 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.waitInitialized in class AbstractStateMachineTableProcedure<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>protected Procedure.LockState acquireLock(MasterProcedureEnv env)
ProcedureProcedure.execute(Object). It calls Procedure.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, see Procedure.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:
acquireLock in class AbstractStateMachineTableProcedure<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState>private boolean prepareCreate(MasterProcedureEnv env) throws IOException
IOExceptionprivate void preCreate(MasterProcedureEnv env) throws IOException, InterruptedException
IOExceptionInterruptedExceptionprivate void postCreate(MasterProcedureEnv env) throws IOException, InterruptedException
IOExceptionInterruptedExceptionprotected static List<RegionInfo> createFsLayout(MasterProcedureEnv env, TableDescriptor tableDescriptor, List<RegionInfo> newRegions) throws IOException
IOExceptionprotected static List<RegionInfo> createFsLayout(MasterProcedureEnv env, TableDescriptor tableDescriptor, List<RegionInfo> newRegions, CreateTableProcedure.CreateHdfsRegions hdfsRegionHandler) throws IOException
IOExceptionprotected static void moveTempDirectoryToHBaseRoot(MasterProcedureEnv env, TableDescriptor tableDescriptor, org.apache.hadoop.fs.Path tempTableDir) throws IOException
IOExceptionprotected static List<RegionInfo> addTableToMeta(MasterProcedureEnv env, TableDescriptor tableDescriptor, List<RegionInfo> regions) throws IOException
IOExceptionprotected static void setEnablingState(MasterProcedureEnv env, TableName tableName) throws IOException
IOExceptionprotected static void setEnabledState(MasterProcedureEnv env, TableName tableName) throws IOException
IOExceptionprivate static void addRegionsToMeta(MasterProcedureEnv env, TableDescriptor tableDescriptor, List<RegionInfo> regionInfos) throws IOException
IOExceptionprotected static void updateTableDescCache(MasterProcedureEnv env, TableName tableName) throws IOException
IOExceptionprotected boolean shouldWaitClientAck(MasterProcedureEnv env)
ProcedureshouldWaitClientAck in class Procedure<MasterProcedureEnv>env - the environment passed to the ProcedureExecutorRegionInfo getFirstRegionInfo()
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.