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
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
Expand All @@ -48,6 +49,7 @@
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import java.util.stream.Stream;
Expand Down Expand Up @@ -167,9 +169,14 @@ public void writeDbDataToStream(DBCheckpoint checkpoint,
// Files to be excluded from tarball
Map<String, Map<Path, Path>> sstFilesToExclude = normalizeExcludeList(toExcludeList,
checkpoint.getCheckpointLocation(), sstBackupDir);
boolean logTotals = false;
if (sstFilesToExclude.isEmpty()) {
logTotals = true;
}

boolean completed = getFilesForArchive(checkpoint, copyFiles,
hardLinkFiles, sstFilesToExclude, includeSnapshotData(request),
excludedList, sstBackupDir, compactionLogDir);
excludedList, sstBackupDir, compactionLogDir, logTotals);
Map<Path, Path> flatCopyFiles = copyFiles.values().stream().flatMap(map -> map.entrySet().stream())
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
writeFilesToArchive(flatCopyFiles, hardLinkFiles, archiveOutputStream,
Expand Down Expand Up @@ -270,19 +277,24 @@ public File getTmpDir() {

@SuppressWarnings("checkstyle:ParameterNumber")
private boolean getFilesForArchive(DBCheckpoint checkpoint,
Map<String, Map<Path, Path>> copyFiles,
Map<Path, Path> hardLinkFiles,
Map<String, Map<Path, Path>> sstFilesToExclude,
boolean includeSnapshotData,
List<String> excluded,
DirectoryData sstBackupDir,
DirectoryData compactionLogDir)
Map<String, Map<Path, Path>> copyFiles,
Map<Path, Path> hardLinkFiles,
Map<String, Map<Path, Path>> sstFilesToExclude,
boolean includeSnapshotData,
List<String> excluded,
DirectoryData sstBackupDir,
DirectoryData compactionLogDir,
boolean logTotals)
throws IOException {

maxTotalSstSize = getConf().getLong(
OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_KEY,
OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_DEFAULT);

AtomicLong totalSize = new AtomicLong(0L);
AtomicInteger totalFiles = new AtomicInteger(0);
int totalSnapshots = 0;

// Tarball limits are not implemented for processes that don't
// include snapshots. Currently, this is just for recon.
if (!includeSnapshotData) {
Expand All @@ -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()) {

try {
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);
Comment on lines +311 to +323
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;

} catch (Exception e) {
LOG.error("Could not determine estimated size of transfer to " +
"Checkpoint Tarball Stream", e);
}
}

if (!processDir(dir, copyFiles, hardLinkFiles, sstFilesToExclude,
new HashSet<>(), excluded, copySize, null)) {
return false;
Expand All @@ -302,7 +337,7 @@ private boolean getFilesForArchive(DBCheckpoint checkpoint,
}

// Get the snapshot files.
Set<Path> snapshotPaths = waitForSnapshotDirs(checkpoint);
Set<Path> snapshotPaths = getSnapshotDirs(checkpoint, true);
Path snapshotDir = getSnapshotDir();
if (!processDir(snapshotDir, copyFiles, hardLinkFiles, sstFilesToExclude,
snapshotPaths, excluded, copySize, null)) {
Expand All @@ -320,16 +355,44 @@ private boolean getFilesForArchive(DBCheckpoint checkpoint,
hardLinkFiles, sstFilesToExclude,
new HashSet<>(), excluded, copySize,
compactionLogDir.getOriginalDir().toPath());
}

private void estimateTotals(Path dir,
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)

totalFiles.addAndGet(paths
.filter(p -> {
File f = p.toFile();
if (f.isDirectory()) {
subDirs.add(p);
return false;
}
return f.getName().toLowerCase().endsWith(".sst");
})
.map(f -> {
try {
totalSize.addAndGet(Files.size(f));
} catch (IOException e) {
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)


for (Path p: subDirs) {
estimateTotals(p, totalFiles, totalSize);
}
}

/**
* The snapshotInfo table may contain a snapshot that
* doesn't yet exist on the fs, so wait a few seconds for it.
* @param checkpoint Checkpoint containing snapshot entries expected.
* @param waitForDir Wait for dir to exist on fs.
* @return Set of expected snapshot dirs.
*/
private Set<Path> waitForSnapshotDirs(DBCheckpoint checkpoint)
private Set<Path> getSnapshotDirs(DBCheckpoint checkpoint, boolean waitForDir)
throws IOException {

OzoneConfiguration conf = getConf();
Expand All @@ -348,7 +411,9 @@ private Set<Path> waitForSnapshotDirs(DBCheckpoint checkpoint)
while (iterator.hasNext()) {
Table.KeyValue<String, SnapshotInfo> entry = iterator.next();
Path path = Paths.get(getSnapshotPath(conf, entry.getValue()));
waitForDirToExist(path);
if (waitForDir) {
waitForDirToExist(path);
}
snapshotPaths.add(path);
}
} finally {
Expand Down Expand Up @@ -552,6 +617,10 @@ private void writeFilesToArchive(
e.getKey().getFileName().toString().toLowerCase().endsWith(".sst")).
collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));

long bytesWritten = 0L;
int filesWritten = 0;
long lastLoggedTime = System.currentTimeMillis();

// Go through each of the files to be copied and add to archive.
for (Map.Entry<Path, Path> entry : filteredCopyFiles.entrySet()) {
Path file = entry.getValue();
Expand All @@ -570,7 +639,18 @@ private void writeFilesToArchive(
fixedFile = f.toString();
}
}
includeFile(entry.getKey().toFile(), fixedFile, archiveOutputStream);
Path f = entry.getKey();
if (!f.toFile().isDirectory()) {
Comment on lines +642 to +643
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.

bytesWritten += Files.size(f);
filesWritten++;
}
includeFile(f.toFile(), fixedFile, archiveOutputStream);
Comment on lines +644 to +647
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.

// Log progress every 30 seconds
if (System.currentTimeMillis() - lastLoggedTime >= 30000) {
LOG.info("Transferred {} KB, #files {} to checkpoint tarball stream...",
bytesWritten / (1024), filesWritten);
lastLoggedTime = System.currentTimeMillis();
Comment on lines +649 to +652
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().

}
}

if (completed) {
Expand All @@ -594,7 +674,11 @@ private void writeFilesToArchive(
}
// Mark tarball completed.
includeRatisSnapshotCompleteFlag(archiveOutputStream);
LOG.info("Completed checkpoint tarball transfer.");
}
LOG.info("Completed transfer of {} KB, #files {} " +
"to checkpoint tarball stream.",
bytesWritten / (1024), filesWritten);
Comment on lines +677 to +681
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.

}

@Nonnull
Expand Down