5555import java .util .ArrayList ;
5656import java .util .List ;
5757import java .util .Map ;
58+ import java .util .Optional ;
5859import java .util .Set ;
5960import java .util .concurrent .TimeUnit ;
60- import org .agrona .collections .MutableLong ;
6161import org .apache .commons .lang3 .StringUtils ;
6262import org .apache .hadoop .conf .Configuration ;
6363import org .apache .hadoop .conf .Configured ;
7878import org .apache .hadoop .hbase .client .Connection ;
7979import org .apache .hadoop .hbase .client .ConnectionFactory ;
8080import org .apache .hadoop .hbase .util .EnvironmentEdgeManager ;
81+ import org .apache .hadoop .hbase .util .Pair ;
8182import org .apache .yetus .audience .InterfaceAudience ;
8283
8384import org .apache .hbase .thirdparty .com .google .common .base .Splitter ;
@@ -736,19 +737,20 @@ private void validatePITRBackupDeletion(String[] backupIds, boolean isForceDelet
736737 }
737738
738739 /**
739- * Identifies tables that rely on the specified backup for PITR. If a table has no other valid
740- * FULL backups that can facilitate recovery to all points within the PITR retention window, it
741- * is added to the dependent list.
742- * @param backupId The backup ID being evaluated.
743- * @return List of tables dependent on the specified backup for PITR.
744- * @throws IOException If backup metadata cannot be retrieved.
740+ * Identifies tables that rely on the specified backup for PITR (Point-In-Time Recovery). A
741+ * table is considered dependent on the backup if it does not have any other valid full backups
742+ * that can cover the PITR window enabled by the specified backup.
743+ * @param backupId The ID of the backup being evaluated for PITR coverage.
744+ * @return A list of tables that are dependent on the specified backup for PITR recovery.
745+ * @throws IOException If there is an error retrieving the backup metadata or backup system
746+ * table.
745747 */
746748 private List <TableName > getTablesDependentOnBackupForPITR (String backupId ) throws IOException {
747749 List <TableName > dependentTables = new ArrayList <>();
748750
749751 try (final BackupSystemTable backupSystemTable = new BackupSystemTable (conn )) {
752+ // Fetch the target backup's info using the backup ID
750753 BackupInfo targetBackup = backupSystemTable .readBackupInfo (backupId );
751-
752754 if (targetBackup == null ) {
753755 throw new IOException ("Backup info not found for backupId: " + backupId );
754756 }
@@ -758,104 +760,121 @@ private List<TableName> getTablesDependentOnBackupForPITR(String backupId) throw
758760 return List .of ();
759761 }
760762
761- // Retrieve the tables with continuous backup enabled and their start times
763+ // Retrieve the tables with continuous backup enabled along with their start times
762764 Map <TableName , Long > continuousBackupStartTimes =
763765 backupSystemTable .getContinuousBackupTableSet ();
764766
765- // Determine the PITR time window
767+ // Calculate the PITR window by fetching configuration and current time
766768 long pitrWindowDays = getConf ().getLong (CONF_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS ,
767769 DEFAULT_CONTINUOUS_BACKUP_PITR_WINDOW_DAYS );
768770 long currentTime = EnvironmentEdgeManager .getDelegate ().currentTime ();
769- final MutableLong pitrMaxStartTime =
770- new MutableLong (currentTime - TimeUnit .DAYS .toMillis (pitrWindowDays ));
771+ final long maxAllowedPITRTime = currentTime - TimeUnit .DAYS .toMillis (pitrWindowDays );
771772
772- // For all tables, determine the earliest (minimum) continuous backup start time.
773- // This represents the actual earliest point-in-time recovery (PITR) timestamp
774- // that can be used, ensuring we do not go beyond the available backup data.
775- long minContinuousBackupStartTime = currentTime ;
773+ // Check each table associated with the target backup
776774 for (TableName table : targetBackup .getTableNames ()) {
777- minContinuousBackupStartTime = Math .min (minContinuousBackupStartTime ,
778- continuousBackupStartTimes .getOrDefault (table , currentTime ));
779- }
780-
781- // The PITR max start time should be the maximum of the calculated minimum continuous backup
782- // start time and the default PITR max start time (based on the configured window).
783- // This ensures that PITR does not extend beyond what is practically possible.
784- pitrMaxStartTime .set (Math .max (minContinuousBackupStartTime , pitrMaxStartTime .longValue ()));
785-
786- for (TableName table : targetBackup .getTableNames ()) {
787- // This backup is not necessary for this table since it doesn't have PITR enabled
775+ // Skip tables without continuous backup enabled
788776 if (!continuousBackupStartTimes .containsKey (table )) {
789777 continue ;
790778 }
791- if (
792- !isValidPITRBackup (targetBackup , table , continuousBackupStartTimes ,
793- pitrMaxStartTime .longValue ())
794- ) {
795- continue ; // This backup is not crucial for PITR of this table
779+
780+ // Calculate the PITR window this backup covers for the table
781+ Optional <Pair <Long , Long >> coveredPitrWindow = getCoveredPitrWindowForTable (targetBackup ,
782+ continuousBackupStartTimes .get (table ), maxAllowedPITRTime , currentTime );
783+
784+ // If this backup does not cover a valid PITR window for the table, skip
785+ if (coveredPitrWindow .isEmpty ()) {
786+ continue ;
796787 }
797788
798- // Check if another valid full backup exists for this table
799- List <BackupInfo > backupHistory = backupSystemTable .getBackupInfos (BackupState .COMPLETE );
800- boolean hasAnotherValidBackup = backupHistory . stream ()
801- . anyMatch ( backup -> ! backup . getBackupId (). equals ( backupId ) && isValidPITRBackup ( backup ,
802- table , continuousBackupStartTimes , pitrMaxStartTime . longValue ()) );
789+ // Check if there is any other valid backup that can cover the PITR window
790+ List <BackupInfo > allBackups = backupSystemTable .getBackupInfos (BackupState .COMPLETE );
791+ boolean hasAnotherValidBackup =
792+ canAnyOtherBackupCover ( allBackups , targetBackup , table , coveredPitrWindow . get () ,
793+ continuousBackupStartTimes . get ( table ), maxAllowedPITRTime , currentTime );
803794
795+ // If no other valid backup exists, add the table to the dependent list
804796 if (!hasAnotherValidBackup ) {
805797 dependentTables .add (table );
806798 }
807799 }
808800 }
801+
809802 return dependentTables ;
810803 }
811804
812805 /**
813- * Determines if a given backup is a valid candidate for Point-In-Time Recovery (PITR) for a
814- * specific table. A valid backup ensures that recovery is possible to any point within the PITR
815- * retention window. A backup qualifies if:
816- * <ul>
817- * <li>It is a FULL backup.</li>
818- * <li>It contains the specified table.</li>
819- * <li>Its completion timestamp is before the PITR retention window start time.</li>
820- * <li>Its completion timestamp is on or after the table’s continuous backup start time.</li>
821- * </ul>
822- * @param backupInfo The backup information being evaluated.
823- * @param tableName The table for which PITR validity is being checked.
824- * @param continuousBackupTables A map of tables to their continuous backup start time.
825- * @param pitrMaxStartTime The maximum allowed start timestamp for PITR eligibility.
826- * @return {@code true} if the backup enables recovery to all valid points in time for the
827- * table; {@code false} otherwise.
806+ * Calculates the PITR (Point-In-Time Recovery) window that the given backup enables for a
807+ * table.
808+ * @param backupInfo Metadata of the backup being evaluated.
809+ * @param continuousBackupStartTime When continuous backups started for the table.
810+ * @param maxAllowedPITRTime The earliest timestamp from which PITR is supported in the
811+ * cluster.
812+ * @param currentTime Current time.
813+ * @return Optional PITR window as a pair (start, end), or empty if backup is not useful for
814+ * PITR.
828815 */
829- private boolean isValidPITRBackup (BackupInfo backupInfo , TableName tableName ,
830- Map <TableName , Long > continuousBackupTables , long pitrMaxStartTime ) {
831- // Only FULL backups are mandatory for PITR
832- if (!BackupType .FULL .equals (backupInfo .getType ())) {
833- return false ;
834- }
816+ private Optional <Pair <Long , Long >> getCoveredPitrWindowForTable (BackupInfo backupInfo ,
817+ long continuousBackupStartTime , long maxAllowedPITRTime , long currentTime ) {
835818
836- // The backup must include the table to be relevant for PITR
837- if (!backupInfo .getTableNames ().contains (tableName )) {
838- return false ;
839- }
819+ long backupStartTs = backupInfo .getStartTs ();
820+ long backupEndTs = backupInfo .getCompleteTs ();
821+ long effectiveStart = Math .max (continuousBackupStartTime , maxAllowedPITRTime );
840822
841- // The backup must have been completed before the PITR retention window starts,
842- // otherwise, it won't be helpful in cases where the recovery point is between
843- // pitrMaxStartTime and the backup completion time.
844- if (backupInfo .getCompleteTs () > pitrMaxStartTime ) {
845- return false ;
823+ if (backupStartTs < continuousBackupStartTime ) {
824+ return Optional .empty ();
846825 }
847826
848- // Retrieve the table's continuous backup start time
849- long continuousBackupStartTime = continuousBackupTables . getOrDefault ( tableName , 0L );
827+ return Optional . of ( Pair . newPair ( Math . max ( backupEndTs , effectiveStart ), currentTime ));
828+ }
850829
851- // The backup must have been started on or after the table’s continuous backup start time,
852- // otherwise, it won't be helpful in few cases because we wouldn't have the WAL entries
853- // between the backup start time and the continuous backup start time.
854- if (backupInfo .getStartTs () < continuousBackupStartTime ) {
855- return false ;
830+ /**
831+ * Checks if any backup (excluding the current backup) can cover the specified PITR window for
832+ * the given table. A backup can cover the PITR window if it fully encompasses the target time
833+ * range specified.
834+ * @param allBackups List of all backups available.
835+ * @param currentBackup The current backup that should not be considered for
836+ * coverage.
837+ * @param table The table for which we need to check backup coverage.
838+ * @param targetWindow A pair representing the target PITR window (start and end
839+ * times).
840+ * @param continuousBackupStartTime When continuous backups started for the table.
841+ * @param maxAllowedPITRTime The earliest timestamp from which PITR is supported in the
842+ * cluster.
843+ * @param currentTime Current time.
844+ * @return {@code true} if any backup (excluding the current one) fully covers the target PITR
845+ * window; {@code false} otherwise.
846+ */
847+ private boolean canAnyOtherBackupCover (List <BackupInfo > allBackups , BackupInfo currentBackup ,
848+ TableName table , Pair <Long , Long > targetWindow , long continuousBackupStartTime ,
849+ long maxAllowedPITRTime , long currentTime ) {
850+
851+ long targetStart = targetWindow .getFirst ();
852+ long targetEnd = targetWindow .getSecond ();
853+
854+ // Iterate through all backups (including the current one)
855+ for (BackupInfo backup : allBackups ) {
856+ // Skip if the backup is not full or doesn't contain the table
857+ if (!BackupType .FULL .equals (backup .getType ())) continue ;
858+ if (!backup .getTableNames ().contains (table )) continue ;
859+
860+ // Skip the current backup itself
861+ if (backup .equals (currentBackup )) continue ;
862+
863+ // Get the covered PITR window for this backup
864+ Optional <Pair <Long , Long >> coveredWindow = getCoveredPitrWindowForTable (backup ,
865+ continuousBackupStartTime , maxAllowedPITRTime , currentTime );
866+
867+ if (coveredWindow .isPresent ()) {
868+ Pair <Long , Long > covered = coveredWindow .get ();
869+
870+ // The backup must fully cover the target window
871+ if (covered .getFirst () <= targetStart && covered .getSecond () >= targetEnd ) {
872+ return true ;
873+ }
874+ }
856875 }
857876
858- return true ;
877+ return false ;
859878 }
860879
861880 @ Override
0 commit comments