DieterDePaepe commented on code in PR #7582:
URL: https://github.com/apache/hbase/pull/7582#discussion_r2913219950
##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java:
##########
Review Comment:
This needs an update due to your changes.
Suggestion:
> Gather all log files that either: 1) are newer than the older timestamps,
but not newer than the newest timestamps. 2) archived logs whose host name does
not occur in newest timestamps.
##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java:
##########
@@ -181,6 +189,50 @@ public void execute() throws IOException {
// set overall backup status: complete. Here we make sure to complete
the backup.
// After this checkpoint, even if entering cancel process, will let the
backup finished
backupInfo.setState(BackupState.COMPLETE);
+
+ // Scan oldlogs for dead/decommissioned hosts and add their max WAL
timestamps
+ // to newTimestamps. This ensures subsequent incremental backups won't
try to back up
+ // WALs that are already covered by this full backup's snapshot.
+ Path walRootDir = CommonFSUtils.getWALRootDir(conf);
+ Path logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
+ Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+ FileSystem fs = walRootDir.getFileSystem(conf);
+
+ List<FileStatus> allLogs = new ArrayList<>();
+ for (FileStatus hostLogDir : fs.listStatus(logDir)) {
+ String host =
BackupUtils.parseHostNameFromLogFile(hostLogDir.getPath());
+ if (host == null) {
+ continue;
+ }
+ allLogs.addAll(Arrays.asList(fs.listStatus(hostLogDir.getPath())));
+ }
+ allLogs.addAll(Arrays.asList(fs.listStatus(oldLogDir)));
+
+ newTimestamps = new HashMap<>();
+
+ for (FileStatus log : allLogs) {
+ if (AbstractFSWALProvider.isMetaFile(log.getPath())) {
+ continue;
+ }
+ String host = BackupUtils.parseHostNameFromLogFile(log.getPath());
+ if (host == null) {
+ continue;
+ }
+ long timestamp = BackupUtils.getCreationTime(log.getPath());
+ Long previousLogRoll = previousLogRollsByHost.get(host);
+ Long latestLogRoll = latestLogRollsByHost.get(host);
+ boolean isInactive = latestLogRoll == null ||
latestLogRoll.equals(previousLogRoll);
+
+ if (isInactive) {
+ long currentTs = newTimestamps.getOrDefault(host, 0L);
+ if (timestamp > currentTs) {
+ newTimestamps.put(host, timestamp);
Review Comment:
Can you add a summarizing javadoc here?
Suggestion, for inactive path:
```
// For a host that was never part of a log roll procedure, or that was
offline during the last log roll:
// newTimestamps takes the maximum WAL file creation timestamp found for
that host in the filesystem
```
for the active path:
```
// For a host that took part of the log roll (and had one or more logs on
the filesystem):
// newTimestamps holds the value of the log roll
```
##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java:
##########
@@ -228,15 +263,6 @@ private List<String> getLogFilesForNewBackup(Map<String,
Long> olderTimestamps,
} else if (currentLogTS > oldTimeStamp) {
resultLogFiles.add(currentLogFile);
}
-
- // It is possible that a host in .oldlogs is an obsolete region server
Review Comment:
Revisiting this comment, and after some discussion with Claude AI, I can
make my remark a bit more concrete:
I think that the `isInactive` check should be updated to use `>=` instead of
`>`:
```
boolean isInactive = earliestTimestampToIncludeInBackup != null
&& earliestTimestampToIncludeInBackup >= latestLogRoll;
```
(This also ensures that a host that was active during last backup, but died
shortly after, is marked as inactive, which makes conceptually sense to be
classified as inactive.)
And that the removed block should be replaced by:
```
if (newTimestamp != null && currentLogTS > newTimestamp) {
newestLogs.add(currentLogFile);
}
```
This covers the scenario where the host is alive during the roll, but dies
during the filesystem scan.
##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java:
##########
@@ -83,12 +83,41 @@ public Map<String, Long> getIncrBackupLogFileMap() throws
IOException {
LOG.info("Execute roll log procedure for incremental backup ...");
BackupUtils.logRoll(conn, backupInfo.getBackupRootDir(), conf);
- newTimestamps = readRegionServerLastLogRollResult();
+ Map<String, Long> newTimestamps = new HashMap<>();
+ Map<String, Long> latestLogRollByHost =
readRegionServerLastLogRollResult();
+ for (Map.Entry<String, Long> entry : latestLogRollByHost.entrySet()) {
+ String host = entry.getKey();
+ long latestLogRoll = entry.getValue();
+ Long earliestTimestampToIncludeInBackup =
previousTimestampMins.get(host);
+
+ boolean isInactive = earliestTimestampToIncludeInBackup != null
+ && earliestTimestampToIncludeInBackup > latestLogRoll;
+
+ if (isInactive) {
+ LOG.debug("Skipping inactive host {} from newTimestamps (boundary={} >
latestLogRoll={})",
+ host, earliestTimestampToIncludeInBackup, latestLogRoll);
+ } else {
+ newTimestamps.put(host, latestLogRoll);
+ }
+ }
logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps,
conf, savedStartCode);
logList = excludeProcV2WALs(logList);
backupInfo.setIncrBackupFileList(logList);
+ // Update boundaries based on WALs that will be backed up
+ for (String logFile : logList) {
+ Path logPath = new Path(logFile);
+ String logHost = BackupUtils.parseHostNameFromLogFile(logPath);
+ if (logHost != null) {
Review Comment:
I would drop this check, since it adds noise to the reading of the code. It
will never trigger since it's guaranteed that `getLogFilesForNewBackup` will
only return logfiles where the host can be parsed.
##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java:
##########
@@ -83,12 +83,41 @@ public Map<String, Long> getIncrBackupLogFileMap() throws
IOException {
LOG.info("Execute roll log procedure for incremental backup ...");
BackupUtils.logRoll(conn, backupInfo.getBackupRootDir(), conf);
- newTimestamps = readRegionServerLastLogRollResult();
+ Map<String, Long> newTimestamps = new HashMap<>();
+ Map<String, Long> latestLogRollByHost =
readRegionServerLastLogRollResult();
+ for (Map.Entry<String, Long> entry : latestLogRollByHost.entrySet()) {
+ String host = entry.getKey();
+ long latestLogRoll = entry.getValue();
+ Long earliestTimestampToIncludeInBackup =
previousTimestampMins.get(host);
+
+ boolean isInactive = earliestTimestampToIncludeInBackup != null
+ && earliestTimestampToIncludeInBackup > latestLogRoll;
+
+ if (isInactive) {
+ LOG.debug("Skipping inactive host {} from newTimestamps (boundary={} >
latestLogRoll={})",
+ host, earliestTimestampToIncludeInBackup, latestLogRoll);
+ } else {
+ newTimestamps.put(host, latestLogRoll);
+ }
+ }
logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps,
conf, savedStartCode);
logList = excludeProcV2WALs(logList);
backupInfo.setIncrBackupFileList(logList);
+ // Update boundaries based on WALs that will be backed up
+ for (String logFile : logList) {
+ Path logPath = new Path(logFile);
+ String logHost = BackupUtils.parseHostNameFromLogFile(logPath);
+ if (logHost != null) {
+ long logTs = BackupUtils.getCreationTime(logPath);
+ Long latestTimestampToIncludeInBackup = newTimestamps.get(logHost);
+ if (latestTimestampToIncludeInBackup == null || logTs >
latestTimestampToIncludeInBackup) {
+ LOG.debug("Updating backup boundary for inactive host {}:
timestamp={}", logHost, logTs);
+ newTimestamps.put(logHost, logTs);
+ }
+ }
+ }
return newTimestamps;
Review Comment:
Similar to my suggestion for FullTableBackupClient, a comment summarizing
what ends up in `newTimestamps` would be useful for future code readers.
##########
hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java:
##########
@@ -181,6 +189,50 @@ public void execute() throws IOException {
// set overall backup status: complete. Here we make sure to complete
the backup.
// After this checkpoint, even if entering cancel process, will let the
backup finished
backupInfo.setState(BackupState.COMPLETE);
+
+ // Scan oldlogs for dead/decommissioned hosts and add their max WAL
timestamps
+ // to newTimestamps. This ensures subsequent incremental backups won't
try to back up
+ // WALs that are already covered by this full backup's snapshot.
+ Path walRootDir = CommonFSUtils.getWALRootDir(conf);
+ Path logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
+ Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+ FileSystem fs = walRootDir.getFileSystem(conf);
+
+ List<FileStatus> allLogs = new ArrayList<>();
+ for (FileStatus hostLogDir : fs.listStatus(logDir)) {
+ String host =
BackupUtils.parseHostNameFromLogFile(hostLogDir.getPath());
+ if (host == null) {
+ continue;
+ }
+ allLogs.addAll(Arrays.asList(fs.listStatus(hostLogDir.getPath())));
+ }
+ allLogs.addAll(Arrays.asList(fs.listStatus(oldLogDir)));
Review Comment:
Noticed that this also affects existing code in IncrementalBackupManager, so
I just logged https://issues.apache.org/jira/browse/HBASE-29984 - no need to
fix it in this PR.
##########
hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupOfflineRS.java:
##########
Review Comment:
Tests look good 👍
--
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]