@InterfaceAudience.Private public class CreateTableProcedure extends StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState> implements TableProcedureInterface
Modifier and Type | Class and Description |
---|---|
protected static interface |
CreateTableProcedure.CreateHdfsRegions |
StateMachineProcedure.Flow
TableProcedureInterface.TableOperationType
Modifier and Type | Field and Description |
---|---|
private AtomicBoolean |
aborted |
private HTableDescriptor |
hTableDescriptor |
private static org.apache.commons.logging.Log |
LOG |
private List<HRegionInfo> |
newRegions |
private ProcedurePrepareLatch |
syncLatch |
private org.apache.hadoop.security.UserGroupInformation |
user |
Constructor and Description |
---|
CreateTableProcedure() |
CreateTableProcedure(MasterProcedureEnv env,
HTableDescriptor hTableDescriptor,
HRegionInfo[] newRegions) |
CreateTableProcedure(MasterProcedureEnv env,
HTableDescriptor hTableDescriptor,
HRegionInfo[] newRegions,
ProcedurePrepareLatch syncLatch) |
Modifier and Type | Method and Description |
---|---|
boolean |
abort(MasterProcedureEnv env)
The abort() call is asynchronous and each procedure must decide how to deal
with that, if they want to be abortable.
|
protected boolean |
acquireLock(MasterProcedureEnv env)
The user should override this method, and try to take a lock if necessary.
|
protected static void |
addRegionsToMeta(MasterProcedureEnv env,
HTableDescriptor hTableDescriptor,
List<HRegionInfo> regionInfos)
Add the specified set of regions to the hbase:meta table.
|
private static List<HRegionInfo> |
addReplicas(MasterProcedureEnv env,
HTableDescriptor hTableDescriptor,
List<HRegionInfo> regions)
Create any replicas for the regions (the default replicas that was
already created is passed to the method)
|
protected static List<HRegionInfo> |
addTableToMeta(MasterProcedureEnv env,
HTableDescriptor hTableDescriptor,
List<HRegionInfo> regions) |
protected static void |
assignRegions(MasterProcedureEnv env,
TableName tableName,
List<HRegionInfo> regions) |
protected static List<HRegionInfo> |
createFsLayout(MasterProcedureEnv env,
HTableDescriptor hTableDescriptor,
List<HRegionInfo> newRegions) |
protected static List<HRegionInfo> |
createFsLayout(MasterProcedureEnv env,
HTableDescriptor hTableDescriptor,
List<HRegionInfo> newRegions,
CreateTableProcedure.CreateHdfsRegions hdfsRegionHandler) |
void |
deserializeStateData(InputStream stream)
Called on store load to allow the user to decode the previously serialized
state.
|
protected StateMachineProcedure.Flow |
executeFromState(MasterProcedureEnv env,
org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState state)
called to perform a single step of the specified 'state' of the procedure
|
protected org.apache.hadoop.hbase.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.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.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.
|
private void |
postCreate(MasterProcedureEnv env) |
private void |
preCreate(MasterProcedureEnv env) |
private boolean |
prepareCreate(MasterProcedureEnv env) |
protected void |
releaseLock(MasterProcedureEnv env)
The user should override this method, and release lock if necessary.
|
protected void |
rollbackState(MasterProcedureEnv env,
org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState state)
called to perform the rollback of the specified state
|
void |
serializeStateData(OutputStream stream)
The user-level code of the procedure may have some state to
persist (e.g.
|
protected void |
setNextState(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState state)
Set the next state for the procedure.
|
protected boolean |
shouldWaitClientAck(MasterProcedureEnv env)
By default, the executor will keep the procedure result around util
the eviction TTL is expired.
|
void |
toStringClassDetails(StringBuilder sb)
Extend the toString() information with the procedure details
e.g.
|
protected static void |
updateTableDescCache(MasterProcedureEnv env,
TableName tableName) |
execute, isYieldAfterExecutionStep, isYieldBeforeExecuteFromState, rollback, toStringState
addStackIndex, beforeReplay, childrenCountDown, compareTo, completionCleanup, convert, convert, createProcedureInfo, doExecute, doRollback, elapsedTime, getException, getLastUpdate, getNonceKey, getOwner, getParentProcId, getProcId, getProcIdHashCode, getResult, getRootProcedureId, getStackIndexes, getStartTime, getState, getTimeout, getTimeRemaining, hasException, hasOwner, hasParent, hasTimeout, incChildrenLatch, isFailed, isFinished, isSuccess, isWaiting, newInstance, removeStackIndex, setAbortFailure, setChildrenLatch, setFailure, setFailure, setNonceKey, setOwner, setParentProcId, setProcId, setResult, setStackIndexes, setStartTime, setState, setTimeout, setTimeoutFailure, toString, toStringClass, toStringDetails, toStringSimpleSB, updateTimestamp, validateClass, wasExecuted
private static final org.apache.commons.logging.Log LOG
private final AtomicBoolean aborted
private final ProcedurePrepareLatch syncLatch
private HTableDescriptor hTableDescriptor
private List<HRegionInfo> newRegions
private org.apache.hadoop.security.UserGroupInformation user
public CreateTableProcedure()
public CreateTableProcedure(MasterProcedureEnv env, HTableDescriptor hTableDescriptor, HRegionInfo[] newRegions)
public CreateTableProcedure(MasterProcedureEnv env, HTableDescriptor hTableDescriptor, HRegionInfo[] newRegions, ProcedurePrepareLatch syncLatch)
protected StateMachineProcedure.Flow executeFromState(MasterProcedureEnv env, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState state) throws InterruptedException
StateMachineProcedure
executeFromState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState>
state
- state to executeInterruptedException
protected void rollbackState(MasterProcedureEnv env, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState state) throws IOException
StateMachineProcedure
rollbackState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState>
state
- state to rollbackIOException
- temporary failure, the rollback will retry laterprotected org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState getState(int stateId)
StateMachineProcedure
getState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState>
stateId
- the ordinal() of the state enum (or state id)protected int getStateId(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState state)
StateMachineProcedure
getStateId
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState>
state
- the state enum objectprotected org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState getInitialState()
StateMachineProcedure
getInitialState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState>
protected void setNextState(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState state)
StateMachineProcedure
setNextState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState>
state
- the state enum objectpublic TableName getTableName()
getTableName
in interface TableProcedureInterface
public TableProcedureInterface.TableOperationType getTableOperationType()
TableProcedureInterface
getTableOperationType
in interface TableProcedureInterface
public boolean abort(MasterProcedureEnv env)
Procedure
abort
in class Procedure<MasterProcedureEnv>
public void toStringClassDetails(StringBuilder sb)
Procedure
toStringClassDetails
in class Procedure<MasterProcedureEnv>
sb
- the string builder to use to append the proc specific informationpublic void serializeStateData(OutputStream stream) throws IOException
Procedure
serializeStateData
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState>
stream
- the stream that will contain the user serialized dataIOException
public void deserializeStateData(InputStream stream) throws IOException
Procedure
deserializeStateData
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState>
stream
- the stream that contains the user serialized dataIOException
protected boolean acquireLock(MasterProcedureEnv env)
Procedure
acquireLock
in class Procedure<MasterProcedureEnv>
protected void releaseLock(MasterProcedureEnv env)
Procedure
releaseLock
in class Procedure<MasterProcedureEnv>
private boolean prepareCreate(MasterProcedureEnv env) throws IOException
IOException
private void preCreate(MasterProcedureEnv env) throws IOException, InterruptedException
IOException
InterruptedException
private void postCreate(MasterProcedureEnv env) throws IOException, InterruptedException
IOException
InterruptedException
protected static List<HRegionInfo> createFsLayout(MasterProcedureEnv env, HTableDescriptor hTableDescriptor, List<HRegionInfo> newRegions) throws IOException
IOException
protected static List<HRegionInfo> createFsLayout(MasterProcedureEnv env, HTableDescriptor hTableDescriptor, List<HRegionInfo> newRegions, CreateTableProcedure.CreateHdfsRegions hdfsRegionHandler) throws IOException
IOException
protected static List<HRegionInfo> addTableToMeta(MasterProcedureEnv env, HTableDescriptor hTableDescriptor, List<HRegionInfo> regions) throws IOException
IOException
private static List<HRegionInfo> addReplicas(MasterProcedureEnv env, HTableDescriptor hTableDescriptor, List<HRegionInfo> regions)
hTableDescriptor
- descriptor to useregions
- default replicasprotected static void assignRegions(MasterProcedureEnv env, TableName tableName, List<HRegionInfo> regions) throws HBaseException, IOException
HBaseException
IOException
protected static void addRegionsToMeta(MasterProcedureEnv env, HTableDescriptor hTableDescriptor, List<HRegionInfo> regionInfos) throws IOException
IOException
protected static void updateTableDescCache(MasterProcedureEnv env, TableName tableName) throws IOException
IOException
protected boolean shouldWaitClientAck(MasterProcedureEnv env)
Procedure
shouldWaitClientAck
in class Procedure<MasterProcedureEnv>
env
- the environment passed to the ProcedureExecutorCopyright © 2007–2019 The Apache Software Foundation. All rights reserved.