@InterfaceAudience.LimitedPrivate(value="Configuration") @InterfaceStability.Unstable public class SnapshotManager extends MasterProcedureManager implements Stoppable
The class provides methods for monitoring in-progress snapshot actions.
Note: Currently there can only be one snapshot being taken at a time over the cluster. This is a simplification in the current implementation.
Modifier and Type | Field and Description |
---|---|
private ProcedureCoordinator |
coordinator |
private ExecutorService |
executorService |
static String |
HBASE_SNAPSHOT_ENABLED
Enable or disable snapshot support
|
static String |
HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS
Wait time before removing a finished sentinel from the in-progress map
NOTE: This is used as a safety auto cleanup.
|
private boolean |
isSnapshotSupported |
private static org.slf4j.Logger |
LOG |
private MasterServices |
master |
static String |
ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION
Name of the operation to use in the controller
|
private Map<TableName,Long> |
restoreTableToProcIdMap |
private org.apache.hadoop.fs.Path |
rootDir |
private ScheduledExecutorService |
scheduleThreadPool |
static int |
SNAPSHOT_POOL_THREADS_DEFAULT
number of current operations running on the master
|
static String |
SNAPSHOT_POOL_THREADS_KEY
Conf key for # of threads used by the SnapshotManager thread pool
|
static long |
SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLS_DEFAULT |
private static int |
SNAPSHOT_WAKE_MILLIS_DEFAULT
By default, check to see if the snapshot is complete every WAKE MILLIS (ms)
|
private static String |
SNAPSHOT_WAKE_MILLIS_KEY
Conf key for # of ms elapsed between checks for snapshot errors while waiting for
completion.
|
private ScheduledFuture<?> |
snapshotHandlerChoreCleanerTask |
private Map<TableName,SnapshotSentinel> |
snapshotHandlers |
private boolean |
stopped |
private ReentrantReadWriteLock |
takingSnapshotLock
Read write lock between taking snapshot and snapshot HFile cleaner.
|
Constructor and Description |
---|
SnapshotManager() |
SnapshotManager(MasterServices master,
ProcedureCoordinator coordinator,
ExecutorService pool,
int sentinelCleanInterval)
Fully specify all necessary components of a snapshot manager.
|
Modifier and Type | Method and Description |
---|---|
void |
checkPermissions(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription desc,
AccessChecker accessChecker,
User user)
Check for required permissions before executing the procedure.
|
void |
checkSnapshotSupport()
Throws an exception if snapshot operations (take a snapshot, restore, clone) are not supported.
|
private void |
checkSnapshotSupport(org.apache.hadoop.conf.Configuration conf,
MasterFileSystem mfs)
Called at startup, to verify if snapshot operation is supported, and to avoid
starting the master if there're snapshots present but the cleaners needed are missing.
|
private void |
cleanupCompletedRestoreInMap()
Remove the procedures that are marked as finished
|
private void |
cleanupSentinels()
Removes "abandoned" snapshot/restore requests.
|
private void |
cleanupSentinels(Map<TableName,SnapshotSentinel> sentinels)
Remove the sentinels that are marked as finished and the completion time
has exceeded the removal timeout.
|
(package private) long |
cloneSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot,
TableDescriptor tableDescriptor,
NonceKey nonceKey,
boolean restoreAcl)
Clone the specified snapshot into a new table.
|
private long |
cloneSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription reqSnapshot,
TableName tableName,
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot,
TableDescriptor snapshotTableDesc,
NonceKey nonceKey,
boolean restoreAcl)
Clone the specified snapshot.
|
void |
deleteSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot)
Delete the specified snapshot
|
void |
execProcedure(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription desc)
Execute a distributed procedure on cluster
|
List<org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription> |
getCompletedSnapshots()
Gets the list of all completed snapshots.
|
private List<org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription> |
getCompletedSnapshots(org.apache.hadoop.fs.Path snapshotDir,
boolean withCpCall)
Gets the list of all completed snapshots.
|
(package private) ProcedureCoordinator |
getCoordinator() |
String |
getProcedureSignature()
Return the unique signature of the procedure.
|
ReadWriteLock |
getTakingSnapshotLock() |
void |
initialize(MasterServices master,
MetricsMaster metricsMaster)
Initialize a globally barriered procedure for master.
|
boolean |
isProcedureDone(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription desc)
Check if the procedure is finished successfully
|
private boolean |
isRestoringTable(TableName tableName)
Verify if the restore of the specified table is in progress.
|
private boolean |
isSnapshotCompleted(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot)
Check to see if the snapshot is one of the currently completed snapshots
Returns true if the snapshot exists in the "completed snapshots folder".
|
boolean |
isSnapshotDone(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription expected)
Check if the specified snapshot is done
|
boolean |
isStopped() |
boolean |
isTakingAnySnapshot()
The snapshot operation processing as following:
1. |
(package private) boolean |
isTakingSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot)
Check to see if there is a snapshot in progress with the same name or on the same table.
|
boolean |
isTakingSnapshot(TableName tableName)
Check to see if the specified table has a snapshot in progress.
|
private void |
prepareToTakeSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot)
Check to make sure that we are OK to run the passed snapshot.
|
private SnapshotSentinel |
removeSentinelIfFinished(Map<TableName,SnapshotSentinel> sentinels,
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot)
Return the handler if it is currently live and has the same snapshot target name.
|
private void |
resetTempDir()
Cleans up any snapshots in the snapshot/.tmp directory that were left from failed
snapshot attempts.
|
long |
restoreOrCloneSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription reqSnapshot,
NonceKey nonceKey,
boolean restoreAcl)
Restore or Clone the specified snapshot
|
private long |
restoreSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot,
TableDescriptor tableDescriptor,
NonceKey nonceKey,
boolean restoreAcl)
Restore the specified snapshot.
|
private long |
restoreSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription reqSnapshot,
TableName tableName,
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot,
TableDescriptor snapshotTableDesc,
NonceKey nonceKey,
boolean restoreAcl)
Restore the specified snapshot.
|
void |
setSnapshotHandlerForTesting(TableName tableName,
SnapshotSentinel handler)
Set the handler for the current snapshot
|
private void |
snapshotDisabledTable(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot)
Take a snapshot of a disabled table.
|
private void |
snapshotEnabledTable(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot)
Take a snapshot of an enabled table.
|
private void |
snapshotTable(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot,
TakeSnapshotHandler handler)
Take a snapshot using the specified handler.
|
void |
stop(String why)
Stop this service.
|
void |
takeSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot)
Take a snapshot based on the enabled/disabled state of the table.
|
private void |
takeSnapshotInternal(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) |
private org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription |
toSnapshotDescription(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription desc) |
execProcedureWithRet
equals, hashCode
private static final org.slf4j.Logger LOG
private static final int SNAPSHOT_WAKE_MILLIS_DEFAULT
public static final String HBASE_SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLIS
public static final long SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT_MILLS_DEFAULT
public static final String HBASE_SNAPSHOT_ENABLED
private static final String SNAPSHOT_WAKE_MILLIS_KEY
public static final String ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION
public static final String SNAPSHOT_POOL_THREADS_KEY
public static final int SNAPSHOT_POOL_THREADS_DEFAULT
private boolean stopped
private MasterServices master
private ProcedureCoordinator coordinator
private boolean isSnapshotSupported
private final Map<TableName,SnapshotSentinel> snapshotHandlers
private final ScheduledExecutorService scheduleThreadPool
private ScheduledFuture<?> snapshotHandlerChoreCleanerTask
private Map<TableName,Long> restoreTableToProcIdMap
private org.apache.hadoop.fs.Path rootDir
private ExecutorService executorService
private ReentrantReadWriteLock takingSnapshotLock
public SnapshotManager()
SnapshotManager(MasterServices master, ProcedureCoordinator coordinator, ExecutorService pool, int sentinelCleanInterval) throws IOException, UnsupportedOperationException
master
- services for the master where the manager is runningcoordinator
- procedure coordinator instance. exposed for testing.pool
- HBase ExecutorServcie instance, exposed for testing.IOException
UnsupportedOperationException
public List<org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription> getCompletedSnapshots() throws IOException
IOException
- File system exceptionprivate List<org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription> getCompletedSnapshots(org.apache.hadoop.fs.Path snapshotDir, boolean withCpCall) throws IOException
snapshotDir
- snapshot directorywithCpCall
- Whether to call CP hooksIOException
- File system exceptionprivate void resetTempDir() throws IOException
IOException
- if we can't reach the filesystempublic void deleteSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) throws IOException
snapshot
- SnapshotDoesNotExistException
- If the specified snapshot does not exist.IOException
- For filesystem IOExceptionspublic boolean isSnapshotDone(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription expected) throws IOException
expected
- IOException
- IOException if error from HDFS or RPCUnknownSnapshotException
- if snapshot is invalid or does not exist.boolean isTakingSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot)
snapshot
- description of the snapshot being checked.public boolean isTakingSnapshot(TableName tableName)
tableName
- name of the table being snapshotted.private void prepareToTakeSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) throws HBaseSnapshotException
snapshot
- description of the snapshot we want to startHBaseSnapshotException
- if the filesystem could not be prepared to start the snapshotprivate void snapshotDisabledTable(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) throws IOException
snapshot
- description of the snapshot to take. Modified to be SnapshotProtos.SnapshotDescription.Type.DISABLED
.IOException
- if the snapshot could not be started or filesystem for snapshot
temporary directory could not be determinedprivate void snapshotEnabledTable(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) throws IOException
snapshot
- description of the snapshot to take.IOException
- if the snapshot could not be started or filesystem for snapshot
temporary directory could not be determinedprivate void snapshotTable(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot, TakeSnapshotHandler handler) throws IOException
snapshot
- the snapshot descriptionhandler
- the snapshot handlerIOException
public ReadWriteLock getTakingSnapshotLock()
public boolean isTakingAnySnapshot()
public void takeSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) throws IOException
snapshot
- HBaseSnapshotException
- when a snapshot specific exception occurs.IOException
- when some sort of generic IO exception occurs.private void takeSnapshotInternal(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) throws IOException
IOException
public void setSnapshotHandlerForTesting(TableName tableName, SnapshotSentinel handler)
Exposed for TESTING
tableName
- handler
- handler the master should use
TODO get rid of this if possible, repackaging, modify tests.ProcedureCoordinator getCoordinator()
private boolean isSnapshotCompleted(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) throws IOException
snapshot
- expected snapshot to checkFileSystem
, false if is
not storedIOException
- if the filesystem throws an unexpected exception,IllegalArgumentException
- if snapshot name is invalid.private long cloneSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription reqSnapshot, TableName tableName, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot, TableDescriptor snapshotTableDesc, NonceKey nonceKey, boolean restoreAcl) throws IOException
reqSnapshot
- Snapshot Descriptor from requesttableName
- table to clonesnapshot
- Snapshot DescriptorsnapshotTableDesc
- Table DescriptornonceKey
- unique identifier to prevent duplicated RPCIOException
long cloneSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot, TableDescriptor tableDescriptor, NonceKey nonceKey, boolean restoreAcl) throws HBaseSnapshotException
snapshot
- Snapshot DescriptortableDescriptor
- Table Descriptor of the table to createnonceKey
- unique identifier to prevent duplicated RPCHBaseSnapshotException
public long restoreOrCloneSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription reqSnapshot, NonceKey nonceKey, boolean restoreAcl) throws IOException
reqSnapshot
- nonceKey
- unique identifier to prevent duplicated RPCIOException
private long restoreSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription reqSnapshot, TableName tableName, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot, TableDescriptor snapshotTableDesc, NonceKey nonceKey, boolean restoreAcl) throws IOException
reqSnapshot
- Snapshot Descriptor from requesttableName
- table to restoresnapshot
- Snapshot DescriptorsnapshotTableDesc
- Table DescriptornonceKey
- unique identifier to prevent duplicated RPCrestoreAcl
- true to restore acl of snapshotIOException
private long restoreSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot, TableDescriptor tableDescriptor, NonceKey nonceKey, boolean restoreAcl) throws HBaseSnapshotException
snapshot
- Snapshot DescriptortableDescriptor
- Table DescriptornonceKey
- unique identifier to prevent duplicated RPCrestoreAcl
- true to restore acl of snapshotHBaseSnapshotException
private boolean isRestoringTable(TableName tableName)
tableName
- table under restoreprivate SnapshotSentinel removeSentinelIfFinished(Map<TableName,SnapshotSentinel> sentinels, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot)
sentinels
- live handlerssnapshot
- snapshot descriptionprivate void cleanupSentinels()
private void cleanupSentinels(Map<TableName,SnapshotSentinel> sentinels)
sentinels
- map of sentinels to cleanprivate void cleanupCompletedRestoreInMap()
public void stop(String why)
Stoppable
public boolean isStopped()
isStopped
in interface Stoppable
Stoppable.stop(String)
has been closed.public void checkSnapshotSupport() throws UnsupportedOperationException
UnsupportedOperationException
- if snapshot are not supportedprivate void checkSnapshotSupport(org.apache.hadoop.conf.Configuration conf, MasterFileSystem mfs) throws IOException, UnsupportedOperationException
conf
- The Configuration
object to usemfs
- The MasterFileSystem to useIOException
- in case of file-system operation failureUnsupportedOperationException
- in case cleaners are missing and
there're snapshot in the systempublic void initialize(MasterServices master, MetricsMaster metricsMaster) throws org.apache.zookeeper.KeeperException, IOException, UnsupportedOperationException
MasterProcedureManager
initialize
in class MasterProcedureManager
master
- Master service interfaceorg.apache.zookeeper.KeeperException
IOException
UnsupportedOperationException
public String getProcedureSignature()
ProcedureManager
getProcedureSignature
in class ProcedureManager
public void execProcedure(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription desc) throws IOException
MasterProcedureManager
execProcedure
in class MasterProcedureManager
desc
- Procedure descriptionIOException
public void checkPermissions(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription desc, AccessChecker accessChecker, User user) throws IOException
MasterProcedureManager
checkPermissions
in class MasterProcedureManager
IOException
- if permissions requirements are not met.public boolean isProcedureDone(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription desc) throws IOException
MasterProcedureManager
isProcedureDone
in class MasterProcedureManager
desc
- Procedure descriptionIOException
private org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription toSnapshotDescription(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription desc) throws IOException
IOException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.