anmolnar commented on code in PR #6922:
URL: https://github.com/apache/hbase/pull/6922#discussion_r2058919214
##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java:
##########
@@ -757,104 +759,127 @@ private List<TableName>
getTablesDependentOnBackupForPITR(String backupId) throw
return List.of();
}
- // Retrieve the tables with continuous backup enabled and their start
times
+ // Retrieve the tables with continuous backup enabled along with their
start times
Map<TableName, Long> continuousBackupStartTimes =
backupSystemTable.getContinuousBackupTableSet();
- // Determine the PITR time window
+ // Calculate the PITR window by fetching configuration and current time
long pitrWindowDays =
getConf().getLong(CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS,
DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS);
long currentTime = EnvironmentEdgeManager.getDelegate().currentTime();
- final MutableLong pitrMaxStartTime =
- new MutableLong(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.set(Math.max(minContinuousBackupStartTime,
pitrMaxStartTime.longValue()));
+ final long maxAllowedPITRTime = currentTime -
TimeUnit.DAYS.toMillis(pitrWindowDays);
+ // Check each table associated with the target backup
for (TableName table : targetBackup.getTableNames()) {
- // This backup is not necessary for this table since it doesn't have
PITR enabled
+ // Skip tables without continuous backup enabled
if (!continuousBackupStartTimes.containsKey(table)) {
continue;
}
- if (
- !isValidPITRBackup(targetBackup, table, continuousBackupStartTimes,
- pitrMaxStartTime.longValue())
- ) {
- continue; // This backup is not crucial for PITR of this table
+
+ // Calculate the PITR window this backup covers for the table
+ Optional<Pair<Long, Long>> coveredPitrWindow =
getCoveredPitrWindowForTable(targetBackup,
+ continuousBackupStartTimes.get(table), maxAllowedPITRTime,
currentTime);
+
+ // If this backup does not cover a valid PITR window for the table,
skip
+ if (coveredPitrWindow.isEmpty()) {
+ continue;
}
- // Check if another valid full backup exists for this table
- List<BackupInfo> backupHistory =
backupSystemTable.getBackupInfos(BackupState.COMPLETE);
- boolean hasAnotherValidBackup = backupHistory.stream()
- .anyMatch(backup -> !backup.getBackupId().equals(backupId) &&
isValidPITRBackup(backup,
- table, continuousBackupStartTimes,
pitrMaxStartTime.longValue()));
+ System.out.println("Backup [" + targetBackup.getBackupId()
+ + "] covered PITR window for table [" + table + "]: " +
coveredPitrWindow);
Review Comment:
Are these stdout prints added to help development for now?
I don't think we should output it to the user.
##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java:
##########
@@ -757,104 +759,127 @@ private List<TableName>
getTablesDependentOnBackupForPITR(String backupId) throw
return List.of();
}
- // Retrieve the tables with continuous backup enabled and their start
times
+ // Retrieve the tables with continuous backup enabled along with their
start times
Map<TableName, Long> continuousBackupStartTimes =
backupSystemTable.getContinuousBackupTableSet();
- // Determine the PITR time window
+ // Calculate the PITR window by fetching configuration and current time
long pitrWindowDays =
getConf().getLong(CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS,
DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS);
long currentTime = EnvironmentEdgeManager.getDelegate().currentTime();
- final MutableLong pitrMaxStartTime =
- new MutableLong(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.set(Math.max(minContinuousBackupStartTime,
pitrMaxStartTime.longValue()));
+ final long maxAllowedPITRTime = currentTime -
TimeUnit.DAYS.toMillis(pitrWindowDays);
+ // Check each table associated with the target backup
for (TableName table : targetBackup.getTableNames()) {
- // This backup is not necessary for this table since it doesn't have
PITR enabled
+ // Skip tables without continuous backup enabled
if (!continuousBackupStartTimes.containsKey(table)) {
continue;
}
- if (
- !isValidPITRBackup(targetBackup, table, continuousBackupStartTimes,
- pitrMaxStartTime.longValue())
- ) {
- continue; // This backup is not crucial for PITR of this table
+
+ // Calculate the PITR window this backup covers for the table
+ Optional<Pair<Long, Long>> coveredPitrWindow =
getCoveredPitrWindowForTable(targetBackup,
+ continuousBackupStartTimes.get(table), maxAllowedPITRTime,
currentTime);
+
+ // If this backup does not cover a valid PITR window for the table,
skip
+ if (coveredPitrWindow.isEmpty()) {
+ continue;
}
- // Check if another valid full backup exists for this table
- List<BackupInfo> backupHistory =
backupSystemTable.getBackupInfos(BackupState.COMPLETE);
- boolean hasAnotherValidBackup = backupHistory.stream()
- .anyMatch(backup -> !backup.getBackupId().equals(backupId) &&
isValidPITRBackup(backup,
- table, continuousBackupStartTimes,
pitrMaxStartTime.longValue()));
+ System.out.println("Backup [" + targetBackup.getBackupId()
+ + "] covered PITR window for table [" + table + "]: " +
coveredPitrWindow);
+
+ // Check if there is any other valid backup that can cover the PITR
window
+ List<BackupInfo> allBackups =
backupSystemTable.getBackupInfos(BackupState.COMPLETE);
+ boolean hasAnotherValidBackup =
+ canAnyOtherBackupCover(allBackups, targetBackup, table,
coveredPitrWindow.get(),
+ continuousBackupStartTimes.get(table), maxAllowedPITRTime,
currentTime);
+ // If no other valid backup exists, add the table to the dependent
list
if (!hasAnotherValidBackup) {
dependentTables.add(table);
}
}
}
+
return dependentTables;
}
/**
- * Determines if a given backup is a valid candidate for Point-In-Time
Recovery (PITR) for a
- * specific table. A valid backup ensures that recovery is possible to any
point within the PITR
- * retention window. A backup qualifies if:
- * <ul>
- * <li>It is a FULL backup.</li>
- * <li>It contains the specified table.</li>
- * <li>Its completion timestamp is before the PITR retention window start
time.</li>
- * <li>Its completion timestamp is on or after the table’s continuous
backup start time.</li>
- * </ul>
- * @param backupInfo The backup information being evaluated.
- * @param tableName The table for which PITR validity is
being checked.
- * @param continuousBackupTables A map of tables to their continuous
backup start time.
- * @param pitrMaxStartTime The maximum allowed start timestamp for
PITR eligibility.
- * @return {@code true} if the backup enables recovery to all valid points
in time for the
- * table; {@code false} otherwise.
+ * Calculates the PITR (Point-In-Time Recovery) window that the given
backup enables for a
+ * table.
+ * @param backupInfo Metadata of the backup being evaluated.
+ * @param continuousBackupStartTime When continuous backups started for
the table.
+ * @param maxAllowedPITRTime The earliest timestamp from which PITR
is supported in the
+ * cluster.
+ * @param currentTime Current time.
+ * @return Optional PITR window as a pair (start, end), or empty if backup
is not useful for
+ * PITR.
*/
- private boolean isValidPITRBackup(BackupInfo backupInfo, TableName
tableName,
- Map<TableName, Long> continuousBackupTables, long pitrMaxStartTime) {
- // Only FULL backups are mandatory for PITR
- if (!BackupType.FULL.equals(backupInfo.getType())) {
- return false;
- }
+ private Optional<Pair<Long, Long>> getCoveredPitrWindowForTable(BackupInfo
backupInfo,
+ long continuousBackupStartTime, long maxAllowedPITRTime, long
currentTime) {
- // The backup must include the table to be relevant for PITR
- if (!backupInfo.getTableNames().contains(tableName)) {
- return false;
- }
+ long backupStartTs = backupInfo.getStartTs();
+ long backupEndTs = backupInfo.getCompleteTs();
+ long effectiveStart = Math.max(continuousBackupStartTime,
maxAllowedPITRTime);
- // The backup must have been completed before the PITR retention window
starts,
- // otherwise, it won't be helpful in cases where the recovery point is
between
- // pitrMaxStartTime and the backup completion time.
- if (backupInfo.getCompleteTs() > pitrMaxStartTime) {
- return false;
+ if (backupStartTs < continuousBackupStartTime) {
+ return Optional.empty();
}
- // Retrieve the table's continuous backup start time
- long continuousBackupStartTime =
continuousBackupTables.getOrDefault(tableName, 0L);
+ return Optional.of(Pair.newPair(Math.max(backupEndTs, effectiveStart),
currentTime));
+ }
- // The backup must have been started on or after the table’s continuous
backup start time,
- // otherwise, it won't be helpful in few cases because we wouldn't have
the WAL entries
- // between the backup start time and the continuous backup start time.
- if (backupInfo.getStartTs() < continuousBackupStartTime) {
- return false;
+ /**
+ * Checks if any backup (excluding the current backup) can cover the
specified PITR window for
+ * the given table. A backup can cover the PITR window if it fully
encompasses the target time
+ * range specified.
+ * @param allBackups List of all backups available.
+ * @param currentBackup The current backup that should not be
considered for
+ * coverage.
+ * @param table The table for which we need to check
backup coverage.
+ * @param targetWindow A pair representing the target PITR
window (start and end
+ * times).
+ * @param continuousBackupStartTime When continuous backups started for
the table.
+ * @param maxAllowedPITRTime The earliest timestamp from which PITR
is supported in the
+ * cluster.
+ * @param currentTime Current time.
+ * @return {@code true} if any backup (excluding the current one) fully
covers the target PITR
+ * window; {@code false} otherwise.
+ */
+ private boolean canAnyOtherBackupCover(List<BackupInfo> allBackups,
BackupInfo currentBackup,
+ TableName table, Pair<Long, Long> targetWindow, long
continuousBackupStartTime,
+ long maxAllowedPITRTime, long currentTime) {
+
+ long targetStart = targetWindow.getFirst();
+ long targetEnd = targetWindow.getSecond();
+
+ // Iterate through all backups (including the current one)
+ for (BackupInfo backup : allBackups) {
+ // Skip if the backup is not full or doesn't contain the table
+ if (!BackupType.FULL.equals(backup.getType())) continue;
+ if (!backup.getTableNames().contains(table)) continue;
+
+ // Skip the current backup itself
+ if (backup.equals(currentBackup)) continue;
+
+ // Get the covered PITR window for this backup
+ Optional<Pair<Long, Long>> coveredWindow =
getCoveredPitrWindowForTable(backup,
+ continuousBackupStartTime, maxAllowedPITRTime, currentTime);
+
+ if (coveredWindow.isPresent()) {
+ Pair<Long, Long> covered = coveredWindow.get();
+
+ // The backup must fully cover the target window
+ if (covered.getFirst() <= targetStart && covered.getSecond() >=
targetEnd) {
+ System.out.println("Backup [" + backup.getBackupId() + "] covers
the target window ["
+ + targetStart + ", " + targetEnd + "] with window [" +
covered.getFirst() + ", "
+ + covered.getSecond() + "]");
Review Comment:
Same here.
--
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]