@InterfaceAudience.Private public class ServerCrashProcedure extends StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState> implements ServerProcedureInterface
The procedure flow varies dependent on whether meta is assigned and if we are to split logs.
We come in here after ServerManager has noticed a server has expired. Procedures queued on the rpc should have been notified about fail and should be concurrently getting themselves ready to assign elsewhere.
StateMachineProcedure.Flow
Procedure.LockState
ServerProcedureInterface.ServerOperationType
Modifier and Type | Field and Description |
---|---|
private boolean |
carryingMeta |
private static org.slf4j.Logger |
LOG |
private boolean |
notifiedDeadServer
Whether DeadServer knows that we are processing it.
|
private List<RegionInfo> |
regionsOnCrashedServer
Regions that were on the crashed server.
|
private ServerName |
serverName
Name of the crashed server to process.
|
private boolean |
shouldSplitWal |
stateCount
NO_PROC_ID, NO_TIMEOUT
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 it, if they want to be abortable.
|
protected Procedure.LockState |
acquireLock(MasterProcedureEnv env)
The user should override this method if they need a lock on an Entity.
|
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.ServerCrashState state)
called to perform a single step of the specified 'state' of the procedure
|
private boolean |
filterDefaultMetaRegions(List<RegionInfo> regions) |
protected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState |
getInitialState()
Return the initial state object that will be used for the first call to executeFromState().
|
protected ProcedureMetrics |
getProcedureMetrics(MasterProcedureEnv env)
Override this method to provide procedure specific counters for submitted count, failed
count and time histogram.
|
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.shaded.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.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState state)
Convert the Enum (or more descriptive) state object to an ordinal (or state id).
|
private List<RegionInfo> |
handleRIT(MasterProcedureEnv env,
List<RegionInfo> regions)
Handle any outstanding RIT that are up against this.serverName, the crashed server.
|
boolean |
hasMetaTableRegion() |
private boolean |
isDefaultMetaRegion(RegionInfo hri) |
protected boolean |
isYieldBeforeExecuteFromState(MasterProcedureEnv env,
org.apache.hadoop.hbase.shaded.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 |
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.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState 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 boolean |
shouldWaitClientAck(MasterProcedureEnv env)
By default, the executor will keep the procedure result around util
the eviction TTL is expired.
|
private void |
splitLogs(MasterProcedureEnv env) |
private void |
splitMetaLogs(MasterProcedureEnv env) |
void |
toStringClassDetails(StringBuilder sb)
Extend the toString() information with the procedure details
e.g.
|
addChildProcedure, execute, failIfAborted, getCurrentState, getCurrentStateId, getCycles, isEofState, isRollbackSupported, isYieldAfterExecutionStep, rollback, setNextState, toStringState
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, 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, waitInitialized, wasExecuted
private static final org.slf4j.Logger LOG
private ServerName serverName
private boolean notifiedDeadServer
private List<RegionInfo> regionsOnCrashedServer
private boolean carryingMeta
private boolean shouldSplitWal
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()
protected StateMachineProcedure.Flow executeFromState(MasterProcedureEnv env, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState state) throws ProcedureSuspendedException, ProcedureYieldException
StateMachineProcedure
executeFromState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState>
state
- state to executeProcedureSuspendedException
ProcedureYieldException
private void processMeta(MasterProcedureEnv env) throws IOException
IOException
private boolean filterDefaultMetaRegions(List<RegionInfo> regions)
private boolean isDefaultMetaRegion(RegionInfo hri)
private void splitMetaLogs(MasterProcedureEnv env) throws IOException
IOException
private void splitLogs(MasterProcedureEnv env) throws IOException
IOException
protected void rollbackState(MasterProcedureEnv env, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState state) throws IOException
StateMachineProcedure
rollbackState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState>
state
- state to rollbackIOException
- temporary failure, the rollback will retry laterprotected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState getState(int stateId)
StateMachineProcedure
getState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState>
stateId
- the ordinal() of the state enum (or state id)protected int getStateId(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState state)
StateMachineProcedure
getStateId
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState>
state
- the state enum objectprotected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState getInitialState()
StateMachineProcedure
getInitialState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState>
protected boolean abort(MasterProcedureEnv env)
Procedure
NOTE: abort() is not like Thread.interrupt(). It is just a notification that allows the procedure implementor abort.
abort
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState>
protected Procedure.LockState acquireLock(MasterProcedureEnv env)
Procedure
Procedure.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 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 informationprotected void serializeStateData(ProcedureStateSerializer serializer) throws IOException
Procedure
serializeStateData
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState>
serializer
- stores the serializable stateIOException
protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException
Procedure
deserializeStateData
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState>
serializer
- contains the serialized stateIOException
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.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState state)
isYieldBeforeExecuteFromState
in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.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 ProcedureExecutorprivate List<RegionInfo> handleRIT(MasterProcedureEnv env, List<RegionInfo> regions)
regions
otherwise we have two assigns going on and they will fight over who has lock.
Notify Unassigns. If unable to unassign because server went away, unassigns block waiting
on the below callback from a ServerCrashProcedure before proceeding.regions
- Regions on the Crashed Server.protected ProcedureMetrics getProcedureMetrics(MasterProcedureEnv env)
Procedure
getProcedureMetrics
in class Procedure<MasterProcedureEnv>
env
- The environment passed to the procedure executorCopyright © 2007–2019 The Apache Software Foundation. All rights reserved.