public class ServerCrashProcedure extends StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState> implements ServerProcedureInterface
The procedure flow varies dependent on whether meta is assigned, if we are doing distributed log replay versus distributed log splitting, and if we are to split logs at all.
This procedure asks that all crashed servers get processed equally; we yield after the completion of each successful flow step. We do this so that we do not 'deadlock' waiting on a region assignment so we can replay edits which could happen if a region moved there are edits on two servers for replay.
TODO: ASSIGN and WAIT_ON_ASSIGN (at least) are not idempotent. Revisit when assign is pv2. TODO: We do not have special handling for system tables.
StateMachineProcedure.Flow
ServerProcedureInterface.ServerOperationType
Modifier and Type | Field and Description |
---|---|
private boolean |
carryingMeta |
private int |
cycles
Cycles on same state.
|
static int |
DEFAULT_RETRIES_ON_META |
static int |
DEFAULT_SHORT_WAIT_ON_META |
static int |
DEFAULT_WAIT_ON_RIT |
private boolean |
distributedLogReplay |
static String |
KEY_RETRIES_ON_META
Configuration key to set how many retries to cycle before we give up on meta.
|
static String |
KEY_SHORT_WAIT_ON_META
Configuration key to set how long to wait in ms doing a quick check on meta state.
|
static String |
KEY_WAIT_ON_RIT
Configuration key to set how long to wait in ms on regions in transition.
|
private static org.apache.commons.logging.Log |
LOG |
private static Set<HRegionInfo> |
META_REGION_SET |
private boolean |
notifiedDeadServer
Whether DeadServer knows that we are processing it.
|
private int |
previousState
Ordinal of the previous state.
|
private List<HRegionInfo> |
regionsAssigned
Regions assigned.
|
private Set<HRegionInfo> |
regionsOnCrashedServer
Regions that were on the crashed server.
|
private ServerName |
serverName
Name of the crashed server to process.
|
private boolean |
shouldSplitWal |
Constructor and Description |
---|
ServerCrashProcedure()
Used when deserializing from a procedure store; we'll construct one of these then call
deserializeStateData(InputStream) . |
ServerCrashProcedure(MasterProcedureEnv env,
ServerName serverName,
boolean shouldSplitWal,
boolean carryingMeta)
Call this constructor queuing up a Procedure.
|
Modifier and Type | Method and Description |
---|---|
protected 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.
|
private boolean |
assign(MasterProcedureEnv env,
List<HRegionInfo> hris) |
private List<HRegionInfo> |
calcRegionsToAssign(MasterProcedureEnv env)
Figure out what we need to assign.
|
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.ServerCrashState state)
called to perform a single step of the specified 'state' of the procedure
|
protected org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState |
getInitialState()
Return the initial state object that will be used for the first call to executeFromState().
|
ServerName |
getServerName() |
ServerProcedureInterface.ServerOperationType |
getServerOperationType()
Given an operation type we can take decisions about what to do with pending operations.
|
protected org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState |
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.ServerCrashState state)
Convert the Enum (or more descriptive) state object to an ordinal (or state id).
|
boolean |
hasMetaTableRegion() |
private boolean |
isMetaAssignedQuickTest(MasterProcedureEnv env)
A quick test that hbase:meta is assigned; blocks for short time only.
|
protected boolean |
isYieldBeforeExecuteFromState(MasterProcedureEnv env,
org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState state)
For this procedure, yield at end of each successful flow step so that all crashed servers
can make progress rather than do the default which has each procedure running to completion
before we move to the next.
|
private void |
prepareLogReplay(MasterProcedureEnv env,
Set<HRegionInfo> regions) |
private static boolean |
processDeadRegion(HRegionInfo hri,
AssignmentManager assignmentManager)
Process a dead region from a dead RS.
|
private boolean |
processMeta(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.ServerCrashState 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 boolean |
shouldWaitClientAck(MasterProcedureEnv env)
By default, the executor will keep the procedure result around util
the eviction TTL is expired.
|
(package private) static int |
size(Collection<HRegionInfo> hris) |
private void |
splitLogs(MasterProcedureEnv env) |
private void |
start(MasterProcedureEnv env)
Start processing of crashed server.
|
private void |
throwProcedureYieldException(String msg) |
void |
toStringClassDetails(StringBuilder sb)
Extend the toString() information with the procedure details
e.g.
|
private void |
verifyAndAssignMeta(MasterProcedureEnv env)
If hbase:meta is not assigned already, assign.
|
private void |
verifyAndAssignMetaWithRetries(MasterProcedureEnv env)
If hbase:meta is not assigned already, assign.
|
private boolean |
waitOnAssign(MasterProcedureEnv env,
List<HRegionInfo> hris) |
private boolean |
waitOnRegionToClearRegionsInTransition(AssignmentManager am,
HRegionInfo hri,
int timeout) |
execute, isYieldAfterExecutionStep, rollback, setNextState, 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
public static final String KEY_SHORT_WAIT_ON_META
public static final int DEFAULT_SHORT_WAIT_ON_META
public static final String KEY_RETRIES_ON_META
KEY_SHORT_WAIT_ON_META
milliseconds.public static final int DEFAULT_RETRIES_ON_META
public static final String KEY_WAIT_ON_RIT
public static final int DEFAULT_WAIT_ON_RIT
private static final Set<HRegionInfo> META_REGION_SET
private ServerName serverName
private boolean notifiedDeadServer
private Set<HRegionInfo> regionsOnCrashedServer
private List<HRegionInfo> regionsAssigned
regionsOnCrashedServer
.private boolean distributedLogReplay
private boolean carryingMeta
private boolean shouldSplitWal
private int cycles
private int previousState
public ServerCrashProcedure(MasterProcedureEnv env, ServerName serverName, boolean shouldSplitWal, boolean carryingMeta)
serverName
- Name of the crashed server.shouldSplitWal
- True if we should split WALs as part of crashed server processing.carryingMeta
- True if carrying hbase:meta table region.public ServerCrashProcedure()
deserializeStateData(InputStream)
. Do not use directly.private void throwProcedureYieldException(String msg) throws ProcedureYieldException
ProcedureYieldException
protected StateMachineProcedure.Flow executeFromState(MasterProcedureEnv env, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState state) throws ProcedureYieldException
StateMachineProcedure
executeFromState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState>
state
- state to executeProcedureYieldException
private void start(MasterProcedureEnv env) throws IOException
env
- IOException
private boolean processMeta(MasterProcedureEnv env) throws IOException
env
- IOException
InterruptedException
private boolean waitOnRegionToClearRegionsInTransition(AssignmentManager am, HRegionInfo hri, int timeout) throws InterruptedIOException
InterruptedIOException
private void prepareLogReplay(MasterProcedureEnv env, Set<HRegionInfo> regions) throws IOException
IOException
private void splitLogs(MasterProcedureEnv env) throws IOException
IOException
static int size(Collection<HRegionInfo> hris)
private List<HRegionInfo> calcRegionsToAssign(MasterProcedureEnv env) throws IOException
env
- IOException
private boolean assign(MasterProcedureEnv env, List<HRegionInfo> hris) throws InterruptedIOException
InterruptedIOException
private boolean waitOnAssign(MasterProcedureEnv env, List<HRegionInfo> hris) throws InterruptedIOException
InterruptedIOException
protected void rollbackState(MasterProcedureEnv env, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState state) throws IOException
StateMachineProcedure
rollbackState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState>
state
- state to rollbackIOException
- temporary failure, the rollback will retry laterprotected org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState getState(int stateId)
StateMachineProcedure
getState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState>
stateId
- the ordinal() of the state enum (or state id)protected int getStateId(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState state)
StateMachineProcedure
getStateId
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState>
state
- the state enum objectprotected org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState getInitialState()
StateMachineProcedure
getInitialState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState>
protected boolean abort(MasterProcedureEnv env)
Procedure
abort
in class Procedure<MasterProcedureEnv>
protected boolean acquireLock(MasterProcedureEnv env)
Procedure
acquireLock
in class Procedure<MasterProcedureEnv>
protected void releaseLock(MasterProcedureEnv env)
Procedure
releaseLock
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.ServerCrashState>
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.ServerCrashState>
stream
- the stream that contains the user serialized dataIOException
private static boolean processDeadRegion(HRegionInfo hri, AssignmentManager assignmentManager) throws IOException
hri
- assignmentManager
- IOException
private void verifyAndAssignMetaWithRetries(MasterProcedureEnv env) throws IOException
IOException
private void verifyAndAssignMeta(MasterProcedureEnv env) throws InterruptedException, IOException, org.apache.zookeeper.KeeperException
InterruptedException
IOException
org.apache.zookeeper.KeeperException
private boolean isMetaAssignedQuickTest(MasterProcedureEnv env) throws InterruptedException, IOException
InterruptedException
IOException
public ServerName getServerName()
getServerName
in interface ServerProcedureInterface
public boolean hasMetaTableRegion()
hasMetaTableRegion
in interface ServerProcedureInterface
public ServerProcedureInterface.ServerOperationType getServerOperationType()
ServerProcedureInterface
getServerOperationType
in interface ServerProcedureInterface
protected boolean isYieldBeforeExecuteFromState(MasterProcedureEnv env, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState state)
isYieldBeforeExecuteFromState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState>
state
- the state we are going to execute next.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.