vinayakphegde commented on code in PR #6848:
URL: https://github.com/apache/hbase/pull/6848#discussion_r2030447009


##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java:
##########
@@ -695,12 +707,161 @@ private void executeDeleteListOfBackups(CommandLine 
cmdline) throws IOException
 
     }
 
+    /**
+     * Validates whether the specified backups can be deleted while preserving 
Point-In-Time
+     * Recovery (PITR) capabilities. If a backup is the only remaining full 
backup enabling PITR for
+     * certain tables, deletion is prevented unless forced.
+     * @param backupIds     Array of backup IDs to validate.
+     * @param isForceDelete Flag indicating whether deletion should proceed 
regardless of PITR
+     *                      constraints.
+     * @throws IOException If a backup is essential for PITR and force 
deletion is not enabled.
+     */
+    private void validatePITRBackupDeletion(String[] backupIds, boolean 
isForceDelete)
+      throws IOException {
+      if (!isForceDelete) {
+        for (String backupId : backupIds) {
+          List<TableName> affectedTables = 
getTablesDependentOnBackupForPITR(backupId);
+          if (!affectedTables.isEmpty()) {
+            String errMsg = String.format(
+              "Backup %s is the only FULL backup remaining that enables PITR 
for tables: %s. "
+                + "Use the force option to delete it anyway.",
+              backupId, affectedTables);
+            System.err.println(errMsg);
+            throw new IOException(errMsg);
+          }
+        }
+      }
+    }
+
+    /**
+     * Identifies tables that rely on the specified backup for PITR. If a 
table has no other valid
+     * FULL backups that can facilitate recovery to all points within the PITR 
retention window, it
+     * is added to the dependent list.
+     * @param backupId The backup ID being evaluated.
+     * @return List of tables dependent on the specified backup for PITR.
+     * @throws IOException If backup metadata cannot be retrieved.
+     */
+    private List<TableName> getTablesDependentOnBackupForPITR(String backupId) 
throws IOException {
+      List<TableName> dependentTables = new ArrayList<>();
+
+      try (final BackupSystemTable backupSystemTable = new 
BackupSystemTable(conn)) {
+        List<BackupInfo> backupHistory = 
backupSystemTable.getBackupInfos(BackupState.COMPLETE);
+        BackupInfo targetBackup = backupSystemTable.readBackupInfo(backupId);
+
+        if (targetBackup == null) {
+          throw new IOException("Backup info not found for backupId: " + 
backupId);
+        }
+
+        // Only full backups are mandatory for PITR
+        if (!BackupType.FULL.equals(targetBackup.getType())) {
+          return List.of();
+        }
+
+        // Retrieve the tables with continuous backup enabled and their start 
times
+        Map<TableName, Long> continuousBackupStartTimes =
+          backupSystemTable.getContinuousBackupTableSet();
+
+        // Determine the PITR time window
+        long pitrWindowDays = 
getConf().getLong(CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS,
+          DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS);
+        long currentTime = EnvironmentEdgeManager.getDelegate().currentTime();
+        long pitrMaxStartTime = currentTime - 
TimeUnit.DAYS.toMillis(pitrWindowDays);
+
+        // For all tables, determine the earliest (minimum) continuous backup 
start time.
+        // This represents the actual earliest point-in-time recovery (PITR) 
timestamp
+        // that can be used, ensuring we do not go beyond the available backup 
data.
+        long minContinuousBackupStartTime = currentTime;
+        for (TableName table : targetBackup.getTableNames()) {
+          minContinuousBackupStartTime = Math.min(minContinuousBackupStartTime,
+            continuousBackupStartTimes.getOrDefault(table, currentTime));
+        }
+
+        // The PITR max start time should be the maximum of the calculated 
minimum continuous backup
+        // start time and the default PITR max start time (based on the 
configured window).
+        // This ensures that PITR does not extend beyond what is practically 
possible.
+        pitrMaxStartTime = Math.max(minContinuousBackupStartTime, 
pitrMaxStartTime);
+
+        for (TableName table : targetBackup.getTableNames()) {
+          // This backup is not necessary for this table since it doesn't have 
PITR enabled
+          if (!continuousBackupStartTimes.containsKey(table)) {
+            continue;
+          }
+          if (
+            !isValidPITRBackup(targetBackup, table, 
continuousBackupStartTimes, pitrMaxStartTime)
+          ) {
+            continue; // This backup is not crucial for PITR of this table
+          }
+
+          // Check if another valid full backup exists for this table
+          long finalPitrMaxStartTime = pitrMaxStartTime;

Review Comment:
   Otherwise we'll get this error: `Variable used in lambda expression should 
be final or effectively final`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to