adoroszlai commented on code in PR #8085:
URL: https://github.com/apache/ozone/pull/8085#discussion_r1996603988


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -570,7 +639,18 @@ private void writeFilesToArchive(
           fixedFile = f.toString();
         }
       }
-      includeFile(entry.getKey().toFile(), fixedFile, archiveOutputStream);
+      Path f = entry.getKey();
+      if (!f.toFile().isDirectory()) {

Review Comment:
   Please rename to `path` and store result of `toFile()` in another variable.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -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);

Review Comment:
   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)



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -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) {

Review Comment:
   New parameter `logTotals` is not needed.
   
   ```suggestion
       if (sstFilesToExclude.isEmpty()) {
   ```



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -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);

Review Comment:
   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.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -570,7 +639,18 @@ private void writeFilesToArchive(
           fixedFile = f.toString();
         }
       }
-      includeFile(entry.getKey().toFile(), fixedFile, archiveOutputStream);
+      Path f = entry.getKey();
+      if (!f.toFile().isDirectory()) {
+        bytesWritten += Files.size(f);
+        filesWritten++;
+      }
+      includeFile(f.toFile(), fixedFile, archiveOutputStream);
+      // 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();

Review Comment:
   Please replace with `System.currentTimeMillis` with `Time.monotonicNow()`.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -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);
+    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());

Review Comment:
   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)



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -570,7 +639,18 @@ private void writeFilesToArchive(
           fixedFile = f.toString();
         }
       }
-      includeFile(entry.getKey().toFile(), fixedFile, archiveOutputStream);
+      Path f = entry.getKey();
+      if (!f.toFile().isDirectory()) {
+        bytesWritten += Files.size(f);
+        filesWritten++;
+      }
+      includeFile(f.toFile(), fixedFile, archiveOutputStream);

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



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -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) {
+      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);

Review Comment:
   - 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.
   
   ```java
           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:
   
   ```java
     private static final PathFilter SST_FILE_FILTER = new 
SuffixFileFilter(ROCKSDB_SST_SUFFIX, IOCase.INSENSITIVE);
   ```
   
   and imports:
   
   ```java
   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;
   ```



-- 
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]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to