@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.FlowProcedure.LockStateServerProcedureInterface.ServerOperationType| Modifier and Type | Field and Description | 
|---|---|
| private boolean | carryingMeta | 
| private org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState | currentRunningState | 
| private static org.slf4j.Logger | LOG | 
| private boolean | notifiedDeadServerWhether DeadServer knows that we are processing it. | 
| private List<RegionInfo> | regionsOnCrashedServerRegions that were on the crashed server. | 
| private ServerName | serverNameName of the crashed server to process. | 
| private boolean | shouldSplitWal | 
| private MonitoredTask | status | 
stateCountNO_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. | 
| private void | assignRegions(MasterProcedureEnv env,
             List<RegionInfo> regions)Assign the regions on the crashed RS to other Rses. | 
| private void | cleanupSplitDir(MasterProcedureEnv env) | 
| private Procedure[] | createSplittingWalProcedures(MasterProcedureEnv env,
                            boolean splitMeta) | 
| 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() | 
| 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. | 
| String | getProcName() | 
| (package private) List<RegionInfo> | getRegionsOnCrashedServer(MasterProcedureEnv env) | 
| 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). | 
| boolean | hasMetaTableRegion() | 
| protected boolean | holdLock(MasterProcedureEnv env)Used to keep the procedure lock even when the procedure is yielding or suspended. | 
| private boolean | isDefaultMetaRegion(RegionInfo hri) | 
| boolean | isInRecoverMetaState() | 
| protected boolean | isMatchingRegionLocation(RegionStateNode rsn)Moved out here so can be overridden by the HBCK fix-up SCP to be less strict about what
 it will tolerate as a 'match'. | 
| private boolean | isSplittingDone(MasterProcedureEnv env,
               boolean splitMeta) | 
| 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. | 
| void | toStringClassDetails(StringBuilder sb)Extend the toString() information with the procedure details
 e.g. | 
| (package private) void | updateProgress(boolean updateState) | 
| static void | updateProgress(MasterProcedureEnv env,
              long parentId) | 
| private void | zkCoordinatedSplitLogs(MasterProcedureEnv env)Split logs using 'classic' zk-based coordination. | 
| private void | zkCoordinatedSplitMetaLogs(MasterProcedureEnv env)Split hbase:meta logs using 'classic' zk-based coordination. | 
addChildProcedure, execute, failIfAborted, getCurrentState, getCurrentStateId, getCycles, isEofState, isRollbackSupported, isYieldAfterExecutionStep, isYieldBeforeExecuteFromState, rollback, setNextState, toStringStateaddStackIndex, afterReplay, beforeReplay, bypass, compareTo, completionCleanup, doExecute, doRollback, elapsedTime, getChildrenLatch, getException, getLastUpdate, getNonceKey, getOwner, getParentProcId, getProcId, getProcIdHashCode, 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, setFailure, setFailure, setLastUpdate, setNonceKey, setOwner, setOwner, setParentProcId, setProcId, setResult, setRootProcId, setStackIndexes, setState, setSubmittedTime, setTimeout, setTimeoutFailure, skipPersistence, toString, toStringClass, toStringDetails, toStringSimpleSB, updateMetricsOnFinish, updateMetricsOnSubmit, updateTimestamp, waitInitialized, wasExecutedprivate static final org.slf4j.Logger LOG
private ServerName serverName
private boolean notifiedDeadServer
private List<RegionInfo> regionsOnCrashedServer
private boolean carryingMeta
private boolean shouldSplitWal
private MonitoredTask status
private org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState currentRunningState
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()
public boolean isInRecoverMetaState()
protected StateMachineProcedure.Flow executeFromState(MasterProcedureEnv env, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState state) throws ProcedureSuspendedException, ProcedureYieldException
StateMachineProcedureexecuteFromState in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState>state - state to executeProcedureSuspendedExceptionProcedureYieldExceptionList<RegionInfo> getRegionsOnCrashedServer(MasterProcedureEnv env)
private void cleanupSplitDir(MasterProcedureEnv env)
private boolean isSplittingDone(MasterProcedureEnv env, boolean splitMeta)
private Procedure[] createSplittingWalProcedures(MasterProcedureEnv env, boolean splitMeta) throws IOException
IOExceptionprivate boolean filterDefaultMetaRegions()
private boolean isDefaultMetaRegion(RegionInfo hri)
private void zkCoordinatedSplitMetaLogs(MasterProcedureEnv env) throws IOException
IOExceptioncreateSplittingWalProcedures(MasterProcedureEnv, boolean)private void zkCoordinatedSplitLogs(MasterProcedureEnv env) throws IOException
IOExceptioncreateSplittingWalProcedures(MasterProcedureEnv, boolean)void updateProgress(boolean updateState)
protected void rollbackState(MasterProcedureEnv env, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState state) throws IOException
StateMachineProcedurerollbackState 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)
StateMachineProceduregetState 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)
StateMachineProceduregetStateId 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()
StateMachineProceduregetInitialState in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState>protected boolean abort(MasterProcedureEnv env)
ProcedureNOTE: 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)
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 Procedure<MasterProcedureEnv>protected void releaseLock(MasterProcedureEnv env)
ProcedurereleaseLock in class Procedure<MasterProcedureEnv>public void toStringClassDetails(StringBuilder sb)
ProceduretoStringClassDetails in class Procedure<MasterProcedureEnv>sb - the string builder to use to append the proc specific informationpublic String getProcName()
getProcName in class Procedure<MasterProcedureEnv>protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException
ProcedureserializeStateData in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState>serializer - stores the serializable stateIOExceptionprotected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException
ProceduredeserializeStateData in class StateMachineProcedure<MasterProcedureEnv,org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState>serializer - contains the serialized stateIOExceptionpublic ServerName getServerName()
getServerName in interface ServerProcedureInterfacepublic boolean hasMetaTableRegion()
hasMetaTableRegion in interface ServerProcedureInterfacepublic ServerProcedureInterface.ServerOperationType getServerOperationType()
ServerProcedureInterfacegetServerOperationType in interface ServerProcedureInterfaceprotected boolean shouldWaitClientAck(MasterProcedureEnv env)
ProcedureshouldWaitClientAck in class Procedure<MasterProcedureEnv>env - the environment passed to the ProcedureExecutorprotected boolean isMatchingRegionLocation(RegionStateNode rsn)
rsn matches that of this crashed server.private void assignRegions(MasterProcedureEnv env, List<RegionInfo> regions) throws IOException
IOExceptionprotected ProcedureMetrics getProcedureMetrics(MasterProcedureEnv env)
ProceduregetProcedureMetrics in class Procedure<MasterProcedureEnv>env - The environment passed to the procedure executorprotected boolean holdLock(MasterProcedureEnv env)
ProcedureholdLock in class Procedure<MasterProcedureEnv>public static void updateProgress(MasterProcedureEnv env, long parentId)
Copyright © 2007–2021 The Apache Software Foundation. All rights reserved.