|
22 | 22 | import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_DESC;
|
23 | 23 | import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG;
|
24 | 24 | import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC;
|
| 25 | +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP; |
| 26 | +import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_ENABLE_CONTINUOUS_BACKUP_DESC; |
25 | 27 | import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_IGNORECHECKSUM;
|
26 | 28 | import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_IGNORECHECKSUM_DESC;
|
27 | 29 | import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_KEEP;
|
|
45 | 47 | import java.io.IOException;
|
46 | 48 | import java.net.URI;
|
47 | 49 | import java.util.List;
|
| 50 | +import java.util.Set; |
48 | 51 | import org.apache.commons.lang3.StringUtils;
|
49 | 52 | import org.apache.hadoop.conf.Configuration;
|
50 | 53 | import org.apache.hadoop.conf.Configured;
|
@@ -339,14 +342,64 @@ public void execute() throws IOException {
|
339 | 342 |
|
340 | 343 | boolean ignoreChecksum = cmdline.hasOption(OPTION_IGNORECHECKSUM);
|
341 | 344 |
|
| 345 | + BackupType backupType = BackupType.valueOf(args[1].toUpperCase()); |
| 346 | + List<TableName> tableNameList = null; |
| 347 | + if (tables != null) { |
| 348 | + tableNameList = Lists.newArrayList(BackupUtils.parseTableNames(tables)); |
| 349 | + } |
| 350 | + boolean continuousBackup = cmdline.hasOption(OPTION_ENABLE_CONTINUOUS_BACKUP); |
| 351 | + if (continuousBackup && !BackupType.FULL.equals(backupType)) { |
| 352 | + System.out.println("ERROR: Continuous backup can Only be specified for Full Backup"); |
| 353 | + printUsage(); |
| 354 | + throw new IOException(INCORRECT_USAGE); |
| 355 | + } |
| 356 | + |
| 357 | + /* |
| 358 | + * The `continuousBackup` flag is specified only during the first full backup to initiate |
| 359 | + * continuous WAL replication. After that, it is redundant because the tables are already set |
| 360 | + * up for continuous backup. If the `continuousBackup` flag is not explicitly enabled, we need |
| 361 | + * to determine the backup mode based on the current state of the specified tables: - If all |
| 362 | + * the specified tables are already part of continuous backup, we treat the request as a |
| 363 | + * continuous backup request and proceed accordingly (since these tables are already |
| 364 | + * continuously backed up, no additional setup is needed). - If none of the specified tables |
| 365 | + * are part of continuous backup, we treat the request as a normal full backup without |
| 366 | + * continuous backup. - If the request includes a mix of tables—some with continuous backup |
| 367 | + * enabled and others without—we cannot determine a clear backup strategy. In this case, we |
| 368 | + * throw an error. If all tables are already in continuous backup mode, we explicitly set the |
| 369 | + * `continuousBackup` flag to `true` so that the request is processed using the continuous |
| 370 | + * backup approach rather than the normal full backup flow. |
| 371 | + */ |
| 372 | + if (!continuousBackup && tableNameList != null && !tableNameList.isEmpty()) { |
| 373 | + try (BackupSystemTable backupSystemTable = new BackupSystemTable(conn)) { |
| 374 | + Set<TableName> continuousBackupTableSet = |
| 375 | + backupSystemTable.getContinuousBackupTableSet().keySet(); |
| 376 | + |
| 377 | + boolean allTablesInContinuousBackup = continuousBackupTableSet.containsAll(tableNameList); |
| 378 | + boolean noTablesInContinuousBackup = |
| 379 | + tableNameList.stream().noneMatch(continuousBackupTableSet::contains); |
| 380 | + |
| 381 | + // Ensure that all tables are either fully in continuous backup or not at all |
| 382 | + if (!allTablesInContinuousBackup && !noTablesInContinuousBackup) { |
| 383 | + System.err |
| 384 | + .println("ERROR: Some tables are already in continuous backup, while others are not. " |
| 385 | + + "Cannot mix both in a single request."); |
| 386 | + printUsage(); |
| 387 | + throw new IOException(INCORRECT_USAGE); |
| 388 | + } |
| 389 | + |
| 390 | + // If all tables are already in continuous backup, enable the flag |
| 391 | + if (allTablesInContinuousBackup) { |
| 392 | + continuousBackup = true; |
| 393 | + } |
| 394 | + } |
| 395 | + } |
| 396 | + |
342 | 397 | try (BackupAdminImpl admin = new BackupAdminImpl(conn)) {
|
343 | 398 | BackupRequest.Builder builder = new BackupRequest.Builder();
|
344 |
| - BackupRequest request = builder.withBackupType(BackupType.valueOf(args[1].toUpperCase())) |
345 |
| - .withTableList( |
346 |
| - tables != null ? Lists.newArrayList(BackupUtils.parseTableNames(tables)) : null) |
| 399 | + BackupRequest request = builder.withBackupType(backupType).withTableList(tableNameList) |
347 | 400 | .withTargetRootDir(targetBackupDir).withTotalTasks(workers)
|
348 | 401 | .withBandwidthPerTasks(bandwidth).withNoChecksumVerify(ignoreChecksum)
|
349 |
| - .withBackupSetName(setName).build(); |
| 402 | + .withBackupSetName(setName).withContinuousBackupEnabled(continuousBackup).build(); |
350 | 403 | String backupId = admin.backupTables(request);
|
351 | 404 | System.out.println("Backup session " + backupId + " finished. Status: SUCCESS");
|
352 | 405 | } catch (IOException e) {
|
@@ -400,6 +453,8 @@ protected void printUsage() {
|
400 | 453 | options.addOption(OPTION_YARN_QUEUE_NAME, true, OPTION_YARN_QUEUE_NAME_DESC);
|
401 | 454 | options.addOption(OPTION_DEBUG, false, OPTION_DEBUG_DESC);
|
402 | 455 | options.addOption(OPTION_IGNORECHECKSUM, false, OPTION_IGNORECHECKSUM_DESC);
|
| 456 | + options.addOption(OPTION_ENABLE_CONTINUOUS_BACKUP, false, |
| 457 | + OPTION_ENABLE_CONTINUOUS_BACKUP_DESC); |
403 | 458 |
|
404 | 459 | HelpFormatter helpFormatter = new HelpFormatter();
|
405 | 460 | helpFormatter.setLeftPadding(2);
|
|
0 commit comments