hemantk-12 commented on code in PR #5104:
URL: https://github.com/apache/ozone/pull/5104#discussion_r1271352852


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -507,6 +507,7 @@ public void onCompactionBegin(RocksDB db,
           }
         }
 
+        waitForTarballCreation();

Review Comment:
   I don't think we need to pause `CompactionBeginListener` because these file 
will never be amended on follower. When new compaction happens on follower 
after bootstrap, these files (SST backup files) will just be overridden. You 
could argue if some file never get compacted but I don't think that would be 
the case if it was getting compacted on leader, it would on follower to 
eventually. 
    
   Note: no further compaction will happen on the output files of compaction 
job till the current compaction finishes. 
   
   The other reason is that this pause will not even let the compaction begin 
itself. Not sure we want that.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -175,23 +188,94 @@ public static Set<Path> normalizeExcludeList(List<String> 
toExcludeList,
     return paths;
   }
 
+  @Override
+  // Pauses rocksdb compaction threads while creating copies of
+  // compaction logs and hard links of sst backups.
+  public DBCheckpoint getCheckpoint(Path tmpdir, boolean flush)
+      throws IOException {
+    DBCheckpoint checkpoint;
+
+    // make tmp directories to contain the copies
+    RocksDBCheckpointDiffer differ = getDbStore().getRocksDBCheckpointDiffer();
+    DirectoryData sstBackupDir = new DirectoryData(tmpdir,
+        differ.getSSTBackupDir());
+    DirectoryData compactionLogDir = new DirectoryData(tmpdir,
+        differ.getCompactionLogDir());
+    if (!sstBackupDir.getTmpDir().mkdirs()) {
+      throw new IOException("mkdirs failed: " + sstBackupDir.getTmpDir());
+    }
+    if (!compactionLogDir.getTmpDir().mkdirs()) {
+      throw new IOException("mkdirs failed: " + compactionLogDir.getTmpDir());
+    }
+
+    long startTime = System.currentTimeMillis();
+    long pauseCounter = PAUSE_COUNTER.incrementAndGet();
+
+    // Pause compactions, Copy/link files and get checkpoint.
+    try {
+      LOG.info("Compaction pausing {} started.", pauseCounter);
+      differ.incrementTarballRequestCount();
+      FileUtils.copyDirectory(compactionLogDir.getDir(),

Review Comment:
   QQ: Why are we using different way to copy data from source to tmp dir?
   Is there any difference between `FileUtils.copyDirectory` and 
`OmSnapshotUtils.linkFiles`? To me they do the similar job.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -175,23 +188,94 @@ public static Set<Path> normalizeExcludeList(List<String> 
toExcludeList,
     return paths;
   }
 
+  @Override
+  // Pauses rocksdb compaction threads while creating copies of
+  // compaction logs and hard links of sst backups.
+  public DBCheckpoint getCheckpoint(Path tmpdir, boolean flush)
+      throws IOException {
+    DBCheckpoint checkpoint;
+
+    // make tmp directories to contain the copies
+    RocksDBCheckpointDiffer differ = getDbStore().getRocksDBCheckpointDiffer();
+    DirectoryData sstBackupDir = new DirectoryData(tmpdir,
+        differ.getSSTBackupDir());
+    DirectoryData compactionLogDir = new DirectoryData(tmpdir,
+        differ.getCompactionLogDir());
+    if (!sstBackupDir.getTmpDir().mkdirs()) {
+      throw new IOException("mkdirs failed: " + sstBackupDir.getTmpDir());
+    }
+    if (!compactionLogDir.getTmpDir().mkdirs()) {
+      throw new IOException("mkdirs failed: " + compactionLogDir.getTmpDir());
+    }
+
+    long startTime = System.currentTimeMillis();
+    long pauseCounter = PAUSE_COUNTER.incrementAndGet();
+
+    // Pause compactions, Copy/link files and get checkpoint.
+    try {
+      LOG.info("Compaction pausing {} started.", pauseCounter);
+      differ.incrementTarballRequestCount();
+      FileUtils.copyDirectory(compactionLogDir.getDir(),
+          compactionLogDir.getTmpDir());
+      OmSnapshotUtils.linkFiles(sstBackupDir.getDir(),
+          sstBackupDir.getTmpDir());
+      checkpoint = getDbStore().getCheckpoint(flush);
+    } finally {
+      // Unpause the compaction threads.
+      synchronized (getDbStore().getRocksDBCheckpointDiffer()) {
+        differ.decrementTarballRequestCount();
+        differ.notifyAll();
+        long elapsedTime = System.currentTimeMillis() - startTime;
+        LOG.info("Compaction pausing {} ended. Elapsed ms: {}",
+            pauseCounter, elapsedTime);
+      }
+    }
+
+    return checkpoint;
+  }
+
+
+  // Convenience class for keeping track of the tmp dir's.
+  private static class DirectoryData {
+    private final File dir;

Review Comment:
   nit: may be `sourceDir` or `srcDir` instead of just dir.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -175,23 +188,94 @@ public static Set<Path> normalizeExcludeList(List<String> 
toExcludeList,
     return paths;
   }
 
+  @Override
+  // Pauses rocksdb compaction threads while creating copies of
+  // compaction logs and hard links of sst backups.
+  public DBCheckpoint getCheckpoint(Path tmpdir, boolean flush)
+      throws IOException {
+    DBCheckpoint checkpoint;
+
+    // make tmp directories to contain the copies
+    RocksDBCheckpointDiffer differ = getDbStore().getRocksDBCheckpointDiffer();
+    DirectoryData sstBackupDir = new DirectoryData(tmpdir,
+        differ.getSSTBackupDir());
+    DirectoryData compactionLogDir = new DirectoryData(tmpdir,
+        differ.getCompactionLogDir());
+    if (!sstBackupDir.getTmpDir().mkdirs()) {
+      throw new IOException("mkdirs failed: " + sstBackupDir.getTmpDir());
+    }

Review Comment:
   nit: this can be move to the `DirectoryData` constructor. 



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -175,23 +188,94 @@ public static Set<Path> normalizeExcludeList(List<String> 
toExcludeList,
     return paths;
   }
 
+  @Override
+  // Pauses rocksdb compaction threads while creating copies of
+  // compaction logs and hard links of sst backups.

Review Comment:
   nit: this should be a java doc comment.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -175,23 +188,94 @@ public static Set<Path> normalizeExcludeList(List<String> 
toExcludeList,
     return paths;
   }
 
+  @Override
+  // Pauses rocksdb compaction threads while creating copies of
+  // compaction logs and hard links of sst backups.
+  public DBCheckpoint getCheckpoint(Path tmpdir, boolean flush)
+      throws IOException {
+    DBCheckpoint checkpoint;
+
+    // make tmp directories to contain the copies
+    RocksDBCheckpointDiffer differ = getDbStore().getRocksDBCheckpointDiffer();
+    DirectoryData sstBackupDir = new DirectoryData(tmpdir,
+        differ.getSSTBackupDir());
+    DirectoryData compactionLogDir = new DirectoryData(tmpdir,
+        differ.getCompactionLogDir());
+    if (!sstBackupDir.getTmpDir().mkdirs()) {
+      throw new IOException("mkdirs failed: " + sstBackupDir.getTmpDir());
+    }
+    if (!compactionLogDir.getTmpDir().mkdirs()) {
+      throw new IOException("mkdirs failed: " + compactionLogDir.getTmpDir());
+    }
+
+    long startTime = System.currentTimeMillis();
+    long pauseCounter = PAUSE_COUNTER.incrementAndGet();
+
+    // Pause compactions, Copy/link files and get checkpoint.
+    try {
+      LOG.info("Compaction pausing {} started.", pauseCounter);
+      differ.incrementTarballRequestCount();
+      FileUtils.copyDirectory(compactionLogDir.getDir(),
+          compactionLogDir.getTmpDir());
+      OmSnapshotUtils.linkFiles(sstBackupDir.getDir(),
+          sstBackupDir.getTmpDir());
+      checkpoint = getDbStore().getCheckpoint(flush);
+    } finally {
+      // Unpause the compaction threads.
+      synchronized (getDbStore().getRocksDBCheckpointDiffer()) {
+        differ.decrementTarballRequestCount();
+        differ.notifyAll();
+        long elapsedTime = System.currentTimeMillis() - startTime;
+        LOG.info("Compaction pausing {} ended. Elapsed ms: {}",
+            pauseCounter, elapsedTime);
+      }
+    }
+
+    return checkpoint;
+  }
+

Review Comment:
   nit: please remove the extra line.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServlet.java:
##########
@@ -295,13 +429,27 @@ private boolean processDir(Path dir, Set<Path> copyFiles,
    * @param excluded The list of db files that actually were excluded.
    */
   @VisibleForTesting
-  public static long processFile(Path file, Set<Path> copyFiles,
+  public static long processFile(Path file, Map<Path, Path> copyFiles,
                                  Map<Path, Path> hardLinkFiles,
                                  Set<Path> toExcludeFiles,
-                                 List<String> excluded) throws IOException {
+                                 List<String> excluded, Path destDir)
+      throws IOException {
     long fileSize = 0;
-    if (toExcludeFiles.contains(file)) {
-      excluded.add(file.toString());
+    Path destFile = file;
+
+    // findbugs nonsense

Review Comment:
   :P



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