rmdmattingly commented on code in PR #5868:
URL: https://github.com/apache/hbase/pull/5868#discussion_r1589581275
##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java:
##########
@@ -295,51 +298,31 @@ public ArrayList<BackupImage> getAncestors(BackupInfo
backupInfo) throws IOExcep
.withRootDir(backup.getBackupRootDir()).withTableList(backup.getTableNames())
.withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build();
- // Only direct ancestors for a backup are required and not entire
history of backup for this
- // table resulting in verifying all of the previous backups which is
unnecessary and backup
- // paths need not be valid beyond the lifetime of a backup.
- //
- // RootDir is way of grouping a single backup including one full and
many incremental backups
+ // If the image has a different rootDir, it cannot be an ancestor.
if (!image.getRootDir().equals(backupInfo.getBackupRootDir())) {
continue;
}
- // add the full backup image as an ancestor until the last incremental
backup
- if (backup.getType().equals(BackupType.FULL)) {
- // check the backup image coverage, if previous image could be covered
by the newer ones,
- // then no need to add
- if (!BackupManifest.canCoverImage(ancestors, image)) {
- ancestors.add(image);
- }
+ // The ancestors consist of the most recent FULL backups that cover the
list of tables
+ // required in the new backup and all INCREMENTAL backups that came
after one of those FULL
+ // backups.
+ if (backup.getType().equals(BackupType.INCREMENTAL)) {
Review Comment:
I believe we're just iterating through all backup history at this point —
what if the given incremental backup is not relevant to the `tablesToCover`? I
think that should not be included as an ancestor, and I think that's what the
`BackupManifest#canCoverImage` was hoping to achieve (but failing to due to the
bugs explained in
https://issues.apache.org/jira/browse/HBASE-28562?focusedCommentId=17843008&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17843008
Hyperbolically, on a cluster with hundreds of tables taking incremental
backups everyday, this would quickly balloon into huge BackupManifests that
cannot fit in a reasonably sized heap
##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java:
##########
@@ -295,51 +298,31 @@ public ArrayList<BackupImage> getAncestors(BackupInfo
backupInfo) throws IOExcep
.withRootDir(backup.getBackupRootDir()).withTableList(backup.getTableNames())
.withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build();
- // Only direct ancestors for a backup are required and not entire
history of backup for this
- // table resulting in verifying all of the previous backups which is
unnecessary and backup
- // paths need not be valid beyond the lifetime of a backup.
- //
- // RootDir is way of grouping a single backup including one full and
many incremental backups
+ // If the image has a different rootDir, it cannot be an ancestor.
if (!image.getRootDir().equals(backupInfo.getBackupRootDir())) {
continue;
}
- // add the full backup image as an ancestor until the last incremental
backup
- if (backup.getType().equals(BackupType.FULL)) {
- // check the backup image coverage, if previous image could be covered
by the newer ones,
- // then no need to add
- if (!BackupManifest.canCoverImage(ancestors, image)) {
- ancestors.add(image);
- }
+ // The ancestors consist of the most recent FULL backups that cover the
list of tables
+ // required in the new backup and all INCREMENTAL backups that came
after one of those FULL
+ // backups.
+ if (backup.getType().equals(BackupType.INCREMENTAL)) {
+ ancestors.add(image);
+ LOG.debug("Dependent incremental backup image: {BackupID={}}",
image.getBackupId());
} else {
- // found last incremental backup, if previously added full backup
ancestor images can cover
- // it, then this incremental ancestor is not the dependent of the
current incremental
- // backup, that is to say, this is the backup scope boundary of
current table set.
- // Otherwise, this incremental backup ancestor is the dependent
ancestor of the ongoing
- // incremental backup
- if (BackupManifest.canCoverImage(ancestors, image)) {
- LOG.debug("Met the backup boundary of the current table set:");
- for (BackupImage image1 : ancestors) {
- LOG.debug(" BackupID={}, BackupDir={}", image1.getBackupId(),
image1.getRootDir());
+ if (tablesToCover.removeAll(image.getTableNames())) {
+ ancestors.add(image);
+ LOG.debug("Dependent full backup image: {BackupID={}}",
image.getBackupId());
+
+ if (tablesToCover.isEmpty()) {
+ LOG.debug("Got {} ancestors for the current backup.",
ancestors.size());
+ return ancestors;
}
Review Comment:
💯 Great addition
--
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]