Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-12557. Added progress indicator for checkpoint tarball in leader OM #8085

Open
wants to merge 1 commit into
base: master
Choose a base branch
from

Conversation

SaketaChalamchala
Copy link
Contributor

What changes were proposed in this pull request?

No logs to indicate checkpoint tarball in leader OM.
Added logs to provide an initial estimate of the amount of data, number of files and snapshots to be transferred in the tarball stream.
Added progress indicator logs every 30s detailing # sst files and size of data trasferred per request.

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/HDDS-12557

How was this patch tested?

Manually tested with docker with a progress log interval of 500ms
Leader OM:


INFO om.OMDBCheckpointServlet: Transfer estimates to Checkpoint Tarball Stream - Estimated Data size: 44 MB, Estimated number of SST files: 35654, Estimated number of Snapshots: 150 
INFO om.OMDBCheckpointServlet: Transferred 55833 KB, #files 2022 to checkpoint tarball stream... // Will print every 30s. 
INFO om.OMDBCheckpointServlet: Completed checkpoint tarball transfer.
INFO om.OMDBCheckpointServlet: Completed transfer of 55833 KB, #files 2022 to checkpoint tarball stream. // excluding hard links.

As usual the follower tracks the amount of data transferred
Follower OM:

2025-03-15 01:10:27,757 [om3-InstallSnapshotThread] INFO ratis_snapshot.OmRatisSnapshotProvider: Downloading latest checkpoint from Leader OM om1. Checkpoint: om.db-om1-1742001027751.tar URL: http://om1:9874/dbCheckpoint?includeSnapshotData=true&flushBeforeCheckpoint=true
2025-03-15 01:10:29,713 [om3-InstallSnapshotThread] INFO ratis_snapshot.OmRatisSnapshotProvider: Download completed for 'om.db-om1-1742001027751.tar'. Total size: 62602 KB

@SaketaChalamchala
Copy link
Contributor Author

@hemantk-12 @swamirishi could you please take a look.

Copy link
Contributor

@adoroszlai adoroszlai left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @SaketaChalamchala for the patch.

@@ -292,6 +304,29 @@ private boolean getFilesForArchive(DBCheckpoint checkpoint,
AtomicLong copySize = new AtomicLong(0L);
// Get the active fs files.
Path dir = checkpoint.getCheckpointLocation();

// Log estimated total data transferred on first request.
if (logTotals) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

New parameter logTotals is not needed.

Suggested change
if (logTotals) {
if (sstFilesToExclude.isEmpty()) {

Comment on lines +311 to +323
estimateTotals(dir, totalFiles, totalSize);
if (includeSnapshotData) {
Set<Path> snapshotPaths = getSnapshotDirs(checkpoint, false);
totalSnapshots = snapshotPaths.size();
for (Path snapshotDir: snapshotPaths) {
estimateTotals(snapshotDir, totalFiles, totalSize);
}
}
LOG.info("Transfer estimates to Checkpoint Tarball Stream - " +
"Estimated Data size: {} MB, " +
"Estimated number of SST files: {}, " +
"Estimated number of Snapshots: {}",
totalSize.get() / (1024 * 1024), totalFiles.get(), totalSnapshots);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • We can use Commons IO for getting recursive file count and total size. estimateTotals can be removed.
  • Declare variables like totalSnapshots close to their usage.
  • Log size in same unit as in other places.
        Counters.PathCounters counters = Counters.longPathCounters();
        CountingPathVisitor visitor = new CountingPathVisitor(counters, SST_FILE_FILTER, TRUE);
        Files.walkFileTree(dir, visitor);
        int totalSnapshots = 0;
        if (includeSnapshotData) {
          Set<Path> snapshotPaths = getSnapshotDirs(checkpoint, false);
          totalSnapshots = snapshotPaths.size();
          for (Path snapshotDir: snapshotPaths) {
            Files.walkFileTree(snapshotDir, visitor);
          }
        }
        LOG.info("Estimates for checkpoint tarball stream: size: {} KB, SST files: {}{}",
            counters.getByteCounter().get() / 1024, counters.getFileCounter().get(),
            (includeSnapshotData ? ", snapshots: " + totalSnapshots : "")
        );

with:

  private static final PathFilter SST_FILE_FILTER = new SuffixFileFilter(ROCKSDB_SST_SUFFIX, IOCase.INSENSITIVE);

and imports:

import static org.apache.commons.io.filefilter.TrueFileFilter.TRUE;

import org.apache.commons.io.IOCase;
import org.apache.commons.io.file.Counters;
import org.apache.commons.io.file.CountingPathVisitor;
import org.apache.commons.io.file.PathFilter;
import org.apache.commons.io.filefilter.SuffixFileFilter;

Comment on lines +642 to +643
Path f = entry.getKey();
if (!f.toFile().isDirectory()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please rename to path and store result of toFile() in another variable.

Comment on lines +649 to +652
if (System.currentTimeMillis() - lastLoggedTime >= 30000) {
LOG.info("Transferred {} KB, #files {} to checkpoint tarball stream...",
bytesWritten / (1024), filesWritten);
lastLoggedTime = System.currentTimeMillis();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please replace with System.currentTimeMillis with Time.monotonicNow().

Comment on lines +677 to +681
LOG.info("Completed checkpoint tarball transfer.");
}
LOG.info("Completed transfer of {} KB, #files {} " +
"to checkpoint tarball stream.",
bytesWritten / (1024), filesWritten);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We could indicate in the Completed transfer of ... message whether transfer is really completed or not, and then Completed checkpoint tarball transfer is not needed.

Comment on lines +644 to +647
bytesWritten += Files.size(f);
filesWritten++;
}
includeFile(f.toFile(), fixedFile, archiveOutputStream);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can avoid Files.size lookup by changing includeFile to return the number of bytes it copies.

AtomicInteger totalFiles,
AtomicLong totalSize) throws IOException {
List<Path> subDirs = new ArrayList<>();
Stream<Path> paths = Files.list(dir);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The stream returned by Files.list should be closed, so best using it in try (...).

(but estimateTotals can be completely removed, so not applicable now)

throw new RuntimeException(e);
}
return f.toString();
}).collect(Collectors.toList()).size());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Collecting into a list just to get its size is not needed, Stream has count().

(but estimateTotals can be completely removed, so not applicable now)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants