diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java index 0a12b62..481d518 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java @@ -35,13 +35,13 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.BackupCopyService; import org.apache.hadoop.hbase.backup.BackupInfo; import org.apache.hadoop.hbase.backup.BackupRestoreServerFactory; import org.apache.hadoop.hbase.backup.BackupType; import org.apache.hadoop.hbase.backup.HBackupFileSystem; -import org.apache.hadoop.hbase.backup.BackupCopyService.Type; import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase; import org.apache.hadoop.hbase.backup.BackupInfo.BackupState; import org.apache.hadoop.hbase.backup.impl.BackupException; @@ -71,7 +71,13 @@ public class FullTableBackupProcedure extends StateMachineProcedure implements TableProcedureInterface { private static final Log LOG = LogFactory.getLog(FullTableBackupProcedure.class); - + + private static final String SNAPSHOT_BACKUP_MAX_ATTEMPTS_KEY = "hbase.backup.snapshot.attempts.max"; + private static final int DEFAULT_SNAPSHOT_BACKUP_MAX_ATTEMPTS = 10; + + private static final String SNAPSHOT_BACKUP_ATTEMPTS_DELAY_KEY = "hbase.backup.snapshot.attempts.delay"; + private static final int DEFAULT_SNAPSHOT_BACKUP_ATTEMPTS_DELAY = 10000; + private final AtomicBoolean aborted = new AtomicBoolean(false); private Configuration conf; private String backupId; @@ -560,17 +566,7 @@ public class FullTableBackupProcedure LOG.debug("Unable to delete " + snapshotName, e); } // Kick off snapshot for backup - try { - env.getMasterServices().getSnapshotManager().takeSnapshot(backupSnapshot); - } catch (IOException e) { - LOG.debug("Unable to take snapshot: " + snapshotName, e); - } - long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout( - env.getMasterConfiguration(), - backupSnapshot.getType(), SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME); - BackupServerUtil.waitForSnapshot(backupSnapshot, waitTime, - env.getMasterServices().getSnapshotManager(), env.getMasterConfiguration()); - // set the snapshot name in BackupStatus of this table, only after snapshot success. + snapshotTable(env, backupSnapshot); backupContext.setSnapshotName(tableName, backupSnapshot.getName()); } setNextState(FullTableBackupState.SNAPSHOT_COPY); @@ -621,6 +617,37 @@ public class FullTableBackupProcedure return Flow.HAS_MORE_STATE; } + private void snapshotTable(final MasterProcedureEnv env, SnapshotDescription backupSnapshot) + throws IOException + { + + int maxAttempts = env.getMasterConfiguration().getInt(SNAPSHOT_BACKUP_MAX_ATTEMPTS_KEY, + DEFAULT_SNAPSHOT_BACKUP_MAX_ATTEMPTS); + int delay = env.getMasterConfiguration().getInt(SNAPSHOT_BACKUP_ATTEMPTS_DELAY_KEY, + DEFAULT_SNAPSHOT_BACKUP_ATTEMPTS_DELAY); + int attempts = 0; + + while (attempts++ < maxAttempts) { + try { + env.getMasterServices().getSnapshotManager().takeSnapshot(backupSnapshot); + long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout( + env.getMasterConfiguration(), + backupSnapshot.getType(), SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME); + BackupServerUtil.waitForSnapshot(backupSnapshot, waitTime, + env.getMasterServices().getSnapshotManager(), env.getMasterConfiguration()); + break; + } catch( NotServingRegionException ee) { + LOG.warn("Snapshot attempt "+attempts +" failed for table "+backupSnapshot.getTable() + + ", sleeping for " + delay+"ms", ee); + if(attempts < maxAttempts) { + try { + Thread.sleep(delay); + } catch (InterruptedException e) { + } + } + } + } + } @Override protected void rollbackState(final MasterProcedureEnv env, final FullTableBackupState state) throws IOException {