You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/10/12 02:19:31 UTC

[GitHub] [ozone] smengcl opened a new pull request, #3824: HDDS 7281 dag v2

smengcl opened a new pull request, #3824:
URL: https://github.com/apache/ozone/pull/3824

   - HDDS-6849. OM changes for tracking Snapshot Metadata. (#3582)
   - HDDS-6853. [Snapshot] Implement Create Snapshot API. (#3652)
   - HDDS-6964. [Snapshot] Split out shared "Path based access" code from OM. (#3653)
   - HDDS-7224. Create a new RocksDBCheckpoint Diff utility. (#3755)
   - HDDS-6855. [SNAPSHOTS] Path Based Access for Ozone Snapshots. (#3729)
   - HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction
   - Make `printSnapdiffSSTFiles` return the list of different SST files.
   - `store.getDb().pauseBackgroundWork()` pauses RDB manual checkpointing as well and causes checkpointing to be blocked indefinitely.
   - Applied https://github.com/apache/ozone/pull/3658 for testing
   - Remove `pauseBackgroundWork` / `continueBackgroundWork` from `RocksDatabase` since they are no longer needed.
   - 1. Compaction logs are now named after active RocksDB's latest sequence number; 2. Compaction logs are saved in its own directory; 3. DAG reconstruction logic has been changed accordingly
   - Remove `SnapshotManager`; clean up
   - findbugs
   - checkstyle. except intentional ones (for now)
   - Clean up `TestOMSnapshotDAG`
   - Populate DAG in `CompletedListener`; de-duplicate `onCompactionCompletedListener`
   - Move SST hardlink logic to `BeginListener` for a peace of mind, as in some cases SST files could be deleted before `CompletedListener` is triggered.
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012214628


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -215,253 +305,265 @@ public static void addDebugLevel(Integer level) {
     DEBUG_LEVEL.add(level);
   }
 
-  // Flushes the WAL and Creates a RocksDB checkpoint
-  @SuppressWarnings({"NM_METHOD_NAMING_CONVENTION"})
-  public void createCheckPoint(String dbPathArg, String cpPathArg,
-                               RocksDB rocksDB) {
-    LOG.warn("Creating Checkpoint for RocksDB instance : " +
-        dbPathArg + "in a CheckPoint Location" + cpPathArg);
-    try {
-      rocksDB.flush(new FlushOptions());
-      Checkpoint cp = Checkpoint.create(rocksDB);
-      cp.createCheckpoint(cpPathArg);
-    } catch (RocksDBException e) {
-      throw new RuntimeException(e.getMessage());
+  /**
+   * Append (then flush) to the current compaction log file path.
+   * Note: This does NOT automatically append newline to the log.
+   */
+  private synchronized void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogPath == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Compaction log path is not set. "
+          + "Please check initialization.");
+      throw new RuntimeException("Compaction log path not set");
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogPath),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogPath, e);
     }
   }
 
-  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
-      throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
-            }
-          }
-        };
+  /**
+   * Append a sequence number to the compaction log (roughly) when an Ozone
+   * snapshot (RDB checkpoint) is taken.
+   * @param sequenceNum RDB sequence number
+   */
+  public void appendSequenceNumberToCompactionLog(long sequenceNum) {
+    final String line = COMPACTION_LOG_SEQNUM_LINE_PREFIX + sequenceNum + "\n";
+    appendToCurrentCompactionLog(line);
+  }
 
-    list.add(onCompactionCompletedListener);
+  /**
+   * Takes {@link org.rocksdb.Options}.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
     rocksOptions.setListeners(list);
   }
 
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
 
+  /**
+   * Takes {@link org.rocksdb.DBOptions}.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
 
-  public void setRocksDBForCompactionTracking(Options rocksOptions)
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      Options rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db,final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file);
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix,
-                      numKeys, currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix, numKeys,
-                        UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null,
-                    compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().size() == 0) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          // Create hardlink backups for the SST files that are going
+          // to be deleted after this RDB compaction.
+          for (String file : compactionJobInfo.inputFiles()) {
+            LOG.debug("Creating hard link for '{}'", file);
+            String saveLinkFileName =
+                sstBackupDir + new File(file).getName();
+            Path link = Paths.get(saveLinkFileName);
+            Path srcFile = Paths.get(file);
+            try {
+              Files.createLink(link, srcFile);
+            } catch (IOException e) {
+              LOG.error("Exception in creating hard link for {}", file);
+              throw new RuntimeException("Failed to create hard link", e);
             }
           }
-        };
 
-    list.add(onCompactionCompletedListener);
-    rocksOptions.setListeners(list);
+        }
+      }
+    };
   }
 
-  public RocksDB getRocksDBInstanceWithCompactionTracking(String dbPath)
-      throws RocksDBException {
-    final Options opt = new Options().setCreateIfMissing(true)
-        .setCompressionType(CompressionType.NO_COMPRESSION);
-    opt.setMaxBytesForLevelMultiplier(2);
-    setRocksDBForCompactionTracking(opt);
-    return RocksDB.open(opt, dbPath);
+  /**
+   * Helper function to append the list of SST files to a StringBuilder
+   * for a compaction log entry. Does not append a new line.
+   */
+  private static void appendCompactionLogStringBuilder(List<String> files,
+      StringBuilder sb) {
+
+    Iterator<String> it = files.iterator();
+    while (it.hasNext()) {

Review Comment:
   Good one! Done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012542936


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -215,390 +288,511 @@ public static void addDebugLevel(Integer level) {
     DEBUG_LEVEL.add(level);
   }
 
-  // Flushes the WAL and Creates a RocksDB checkpoint
-  @SuppressWarnings({"NM_METHOD_NAMING_CONVENTION"})
-  public void createCheckPoint(String dbPathArg, String cpPathArg,
-                               RocksDB rocksDB) {
-    LOG.warn("Creating Checkpoint for RocksDB instance : " +
-        dbPathArg + "in a CheckPoint Location" + cpPathArg);
-    try {
-      rocksDB.flush(new FlushOptions());
-      Checkpoint cp = Checkpoint.create(rocksDB);
-      cp.createCheckpoint(cpPathArg);
-    } catch (RocksDBException e) {
-      throw new RuntimeException(e.getMessage());
+  /**
+   * Append (then flush) to the current compaction log file path.
+   * Note: This does NOT automatically append newline to the log.
+   */
+  private synchronized void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogPath == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Compaction log path is not set. "
+          + "Please check initialization.");
+      throw new RuntimeException("Compaction log path not set");
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogPath),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogPath, e);
     }
   }
 
-  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
-      throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
-            }
-          }
-        };
+  /**
+   * Append a sequence number to the compaction log (roughly) when an Ozone
+   * snapshot (RDB checkpoint) is taken.
+   * @param sequenceNum RDB sequence number
+   */
+  public void appendSequenceNumberToCompactionLog(long sequenceNum) {
+    final String line = COMPACTION_LOG_SEQNUM_LINE_PREFIX + sequenceNum + "\n";
+    appendToCurrentCompactionLog(line);
+  }
 
-    list.add(onCompactionCompletedListener);
+  /**
+   * Takes {@link org.rocksdb.Options}.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
     rocksOptions.setListeners(list);
   }
 
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
 
+  /**
+   * Takes {@link org.rocksdb.DBOptions}.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
 
-  public void setRocksDBForCompactionTracking(Options rocksOptions)
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      Options rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db,final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file);
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix,
-                      numKeys, currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix, numKeys,
-                        UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null,
-                    compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().size() == 0) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          // Create hardlink backups for the SST files that are going
+          // to be deleted after this RDB compaction.
+          for (String file : compactionJobInfo.inputFiles()) {
+            LOG.debug("Creating hard link for '{}'", file);
+            String saveLinkFileName =
+                sstBackupDir + new File(file).getName();
+            Path link = Paths.get(saveLinkFileName);
+            Path srcFile = Paths.get(file);
+            try {
+              Files.createLink(link, srcFile);
+            } catch (IOException e) {
+              LOG.error("Exception in creating hard link for {}", file);
+              throw new RuntimeException("Failed to create hard link", e);
             }
           }
-        };
 
-    list.add(onCompactionCompletedListener);
-    rocksOptions.setListeners(list);
+        }
+      }
+    };
   }
 
-  public RocksDB getRocksDBInstanceWithCompactionTracking(String dbPath)
-      throws RocksDBException {
-    final Options opt = new Options().setCreateIfMissing(true)
-        .setCompressionType(CompressionType.NO_COMPRESSION);
-    opt.setMaxBytesForLevelMultiplier(2);
-    setRocksDBForCompactionTracking(opt);
-    return RocksDB.open(opt, dbPath);
+  private AbstractEventListener newCompactionCompletedListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionCompleted(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().isEmpty()) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          final StringBuilder sb = new StringBuilder();
+
+          if (LOG.isDebugEnabled()) {
+            // Print compaction reason for this entry in the log file
+            // e.g. kLevelL0FilesNum / kLevelMaxLevelSize.
+            sb.append(COMPACTION_LOG_COMMENT_LINE_PREFIX)
+                .append(compactionJobInfo.compactionReason())
+                .append('\n');
+          }
+
+          // Mark the beginning of a compaction log
+          sb.append(COMPACTION_LOG_ENTRY_LINE_PREFIX);
+
+          // Trim DB path, only keep the SST file name

Review Comment:
   The full path is highly redundant. IMO there is no need to persist the full path as the SST file can either be in the SST backup dir or in active DB dir. Storing the full path easily doubles the in-memory map memory consumption compared to file name only.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012307907


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java:
##########
@@ -29,60 +31,114 @@
 import java.util.Random;
 import java.util.stream.Collectors;
 
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.DifferSnapshotInfo;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.rocksdb.Checkpoint;
 import org.rocksdb.ColumnFamilyDescriptor;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.ColumnFamilyOptions;
-import org.rocksdb.CompressionType;
 import org.rocksdb.DBOptions;
+import org.rocksdb.FlushOptions;
 import org.rocksdb.LiveFileMetaData;
 import org.rocksdb.Options;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 import org.rocksdb.RocksIterator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
 
-////CHECKSTYLE:OFF
+/**
+ * Test RocksDBCheckpointDiffer basic functionality.
+ */
 public class TestRocksDBCheckpointDiffer {
 
-  private static final String dbPath   = "./rocksdb-data";
-  private static final int NUM_ROW = 25000000;
-  private static final int SNAPSHOT_EVERY_SO_MANY_KEYS = 999999;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRocksDBCheckpointDiffer.class);
+
+  /**
+   * RocksDB path for the test.
+   */
+  private static final String TEST_DB_PATH = "./rocksdb-data";
+  private static final int NUM_ROW = 250000;
+  private static final int SNAPSHOT_EVERY_SO_MANY_KEYS = 49999;
+
+  /**
+   * RocksDB checkpoint path prefix.
+   */
+  private static final String CP_PATH_PREFIX = "rocksdb-cp-";
+  private final ArrayList<DifferSnapshotInfo> snapshots = new ArrayList<>();
+
+  /**
+   * Graph type.
+   */
+  enum GType {
+    FNAME,
+    KEYSIZE,
+    CUMUTATIVE_SIZE
+  }
+
+  @BeforeAll
+  public static void init() {
+    // Set differ log level
+    GenericTestUtils.setLogLevel(RocksDBCheckpointDiffer.getLog(), Level.INFO);
+    // Set test class log level
+    GenericTestUtils.setLogLevel(TestRocksDBCheckpointDiffer.LOG, Level.INFO);
+  }
 
-  // keeps track of all the snapshots created so far.
-  private static int lastSnapshotCounter = 0;
-  private static String lastSnapshotPrefix = "snap_id_";
+  @Test

Review Comment:
   This test doesn't seem to check the results.  Is that going to happen in a future pr?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on PR #3824:
URL: https://github.com/apache/ozone/pull/3824#issuecomment-1276774483

   These lines are confusing to me:  https://github.com/apache/ozone/blob/c7e922fb0c4d4c6c312bd717246c9fd8fde27e6f/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java#L734-L738
   
   I think something like this would be clearer:
   ```
         Set<CompactionNode> currentLevel = new HashSet<>();
         currentLevel.add(infileNode);
         Set<CompactionNode> nextLevel = new HashSet<>();
   ```
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993950723


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -114,9 +113,82 @@ public class RocksDBCheckpointDiffer {
   private static final long UNKNOWN_COMPACTION_GEN = 0;
   private long currentCompactionGen = 0;
 
-  // Something to track all the snapshots created so far.
+  // Something to track all the snapshots created so far. TODO: REMOVE
   private Snapshot[] allSnapshots;
 
+  private String compactionLogParentDir = null;
+  private String compactionLogDir = null;
+
+  // Name of the directory to hold compaction logs (under parent dir)
+  private static final String COMPACTION_LOG_DIR = "compaction-log/";
+
+  // For DB compaction SST DAG persistence and reconstruction
+  // Should be initialized to the latest sequence number
+  private volatile String currentCompactionLogFilename = null;
+
+  private static final String COMPACTION_LOG_FILENAME_SUFFIX = ".log";
+
+  public void setCompactionLogParentDir(String parentDir) {
+    this.compactionLogParentDir = parentDir;
+
+    // Append /
+    if (!compactionLogParentDir.endsWith("/")) {
+      compactionLogParentDir += "/";
+    }
+
+    File pDir = new File(compactionLogParentDir);
+    if (!pDir.exists()) {
+      if (!pDir.mkdir()) {
+        LOG.error("Error creating compaction log parent dir.");
+        return;
+      }
+    }
+
+    compactionLogDir =
+        Paths.get(compactionLogParentDir, COMPACTION_LOG_DIR).toString();
+    File clDir = new File(compactionLogDir);
+    if (!clDir.mkdir()) {
+      LOG.error("Error creating compaction log dir.");
+      return;
+    }
+
+    // TODO: Write a README there explaining what the dir is for
+  }
+
+  private static final int LONG_MAX_STRLEN =
+      String.valueOf(Long.MAX_VALUE).length();
+
+  public void setCompactionLogFilenameBySeqNum(long latestSequenceId) {

Review Comment:
   NIT: this method name doesn't quite convey the functionality to me.  What do you think of something like: "setCurrentCompactionLog()" ?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1007992037


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,38 +539,213 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  private String[] sstTokensRead;
+  private long reconstructionSnapshotGeneration;
+
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   */
+  private synchronized void processCompactionLogLine(String line) {
+
+    LOG.debug("Processing line: {}", line);
+
+    // Skip comments
+    if (line.startsWith("#")) {
+      LOG.debug("Comment line, skipped");
+      return;
+    }
+
+    // Read sequence number as the snapshot generation for the nodes
+    if (line.startsWith(COMPACTION_LOG_SEQNUM_LINE_PREFIX)) {
+      LOG.debug("Reading sequence number as snapshot generation");
+      final String seqNumStr =
+          line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length()).trim();
+      reconstructionSnapshotGeneration = Long.parseLong(seqNumStr);
+      return;
+    }
+
+    if (sstTokensRead == null) {
+      // Store the tokens in the first line
+      sstTokensRead = line.split("\t");
+      LOG.debug("Number of input files: {}", sstTokensRead.length);
+      if (sstTokensRead.length == 0) {
+        // Sanity check. inputFiles should never be empty. outputFiles could.
+        throw new RuntimeException(
+            "Compaction inputFiles list is empty. File is corrupted?");
+      }
+    } else {
+      final String[] outputFilesRead = line.split("\t");
+      LOG.debug("Number of output files: {}", outputFilesRead.length);
+
+      // Populate the compaction DAG
+      populateCompactionDAG(sstTokensRead, outputFilesRead,
+          reconstructionSnapshotGeneration);
+
+      // Reset inputFilesRead to null so
+      sstTokensRead = null;
+    }
+  }
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   * <p>
+   * DO NOT use this function in another context without understanding what it
+   * does, due to the state preserved between calls (in sstTokensRead).
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {

Review Comment:
   Done. Now previous compaction logs will **only** be loaded during OM startup.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012335817


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.freon;
+
+import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
+import org.apache.hadoop.hdds.utils.db.RDBStore;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMStorage;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.DifferSnapshotInfo;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.raftlog.RaftLog;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_S3_VOLUME_NAME_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+
+/**
+ * Tests Freon, with MiniOzoneCluster.
+ */
+public class TestOMSnapshotDAG {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOMSnapshotDAG.class);
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+  private static ObjectStore store;
+  private final File metaDir = OMStorage.getOmDbDir(conf);
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting OZONE_ENABLED = true
+   *
+   */
+  @BeforeAll
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    DatanodeRatisServerConfig ratisServerConfig =
+        conf.getObject(DatanodeRatisServerConfig.class);
+    ratisServerConfig.setRequestTimeOut(Duration.ofSeconds(3));
+    ratisServerConfig.setWatchTimeOut(Duration.ofSeconds(3));
+    conf.setFromObject(ratisServerConfig);
+
+    RatisClientConfig.RaftConfig raftClientConfig =
+        conf.getObject(RatisClientConfig.RaftConfig.class);
+    raftClientConfig.setRpcRequestTimeout(Duration.ofSeconds(3));
+    raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(3));
+    conf.setFromObject(raftClientConfig);
+
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(5).build();
+    cluster.waitForClusterToBeReady();
+
+    store = cluster.getClient().getObjectStore();
+
+    GenericTestUtils.setLogLevel(RaftLog.LOG, Level.INFO);
+    GenericTestUtils.setLogLevel(RaftServer.LOG, Level.INFO);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterAll
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private String getDBCheckpointAbsolutePath(SnapshotInfo snapshotInfo) {
+    return metaDir + OM_KEY_PREFIX +
+        OM_SNAPSHOT_DIR + OM_KEY_PREFIX +
+        OM_DB_NAME + snapshotInfo.getCheckpointDirName();
+  }
+
+  private static String getSnapshotDBKey(String volumeName, String bucketName,
+      String snapshotName) {
+
+    final String dbKeyPrefix = OM_KEY_PREFIX + volumeName +
+        OM_KEY_PREFIX + bucketName;
+    return dbKeyPrefix + OM_KEY_PREFIX + snapshotName;
+  }
+
+  private DifferSnapshotInfo getDifferSnapshotInfo(
+      OMMetadataManager omMetadataManager, String volumeName, String bucketName,
+      String snapshotName) throws IOException {
+
+    final String dbKey = getSnapshotDBKey(volumeName, bucketName, snapshotName);
+    final SnapshotInfo snapshotInfo =
+        omMetadataManager.getSnapshotInfoTable().get(dbKey);
+    String checkpointPath = getDBCheckpointAbsolutePath(snapshotInfo);
+
+    // Use RocksDB transaction sequence number in SnapshotInfo, which is
+    // persisted at the time of snapshot creation, as the snapshot generation
+    return new DifferSnapshotInfo(checkpointPath, snapshotInfo.getSnapshotID(),
+        snapshotInfo.getDbTxSequenceNumber());
+  }
+
+  @Test
+  void testZeroSizeKey()
+      throws IOException, InterruptedException, TimeoutException {
+
+    RandomKeyGenerator randomKeyGenerator =
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "600",
+        "--num-of-threads", "1",
+        "--key-size", "0",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
+    Assertions.assertEquals(600L, randomKeyGenerator.getNumberOfKeysAdded());
+    Assertions.assertEquals(600L,
+        randomKeyGenerator.getSuccessfulValidationCount());
+
+    List<OmVolumeArgs> volList = cluster.getOzoneManager()
+        .listAllVolumes("", "", 10);
+    LOG.debug("List of all volumes: {}", volList);
+    final String volumeName = volList.stream().filter(e ->
+        !e.getVolume().equals(OZONE_S3_VOLUME_NAME_DEFAULT))  // Ignore s3v vol
+        .collect(Collectors.toList()).get(0).getVolume();
+    List<OmBucketInfo> bucketList =
+        cluster.getOzoneManager().listBuckets(volumeName, "", "", 10);
+    LOG.debug("List of all buckets under the first volume: {}", bucketList);
+    final String bucketName = bucketList.get(0).getBucketName();
+
+    // Create snapshot
+    String resp = store.createSnapshot(volumeName, bucketName, "snap1");
+    LOG.debug("Snapshot created: {}", resp);
+
+    final OzoneVolume volume = store.getVolume(volumeName);
+    final OzoneBucket bucket = volume.getBucket(bucketName);
+
+    for (int i = 0; i < 6000; i++) {
+      bucket.createKey("b_" + i, 0).close();
+    }
+
+    // Create another snapshot
+    resp = store.createSnapshot(volumeName, bucketName, "snap3");
+    LOG.debug("Snapshot created: {}", resp);
+
+    // Get snapshot SST diff list
+    OzoneManager ozoneManager = cluster.getOzoneManager();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    RDBStore rdbStore = (RDBStore) omMetadataManager.getStore();
+    RocksDBCheckpointDiffer differ = rdbStore.getRocksDBCheckpointDiffer();
+
+    DifferSnapshotInfo snap1 = getDifferSnapshotInfo(omMetadataManager,
+        volumeName, bucketName, "snap1");
+    DifferSnapshotInfo snap3 = getDifferSnapshotInfo(omMetadataManager,
+        volumeName, bucketName, "snap3");
+
+    // RocksDB does checkpointing in a separate thread, wait for it
+    final File checkpointSnap1 = new File(snap1.getDbPath());
+    GenericTestUtils.waitFor(checkpointSnap1::exists, 2000, 20000);
+    final File checkpointSnap3 = new File(snap3.getDbPath());
+    GenericTestUtils.waitFor(checkpointSnap3::exists, 2000, 20000);
+
+    List<String> actualDiffList = differ.getSSTDiffList(snap3, snap1);
+    LOG.debug("Got diff list: {}", actualDiffList);
+    final List<String> expectedDiffList = Arrays.asList("000059");

Review Comment:
   Ok, it would be good to explain in a comment so reviewers don't spend unnecessary time wondering where it came from.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012309412


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.freon;
+
+import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
+import org.apache.hadoop.hdds.utils.db.RDBStore;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMStorage;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.DifferSnapshotInfo;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.raftlog.RaftLog;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_S3_VOLUME_NAME_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+
+/**
+ * Tests Freon, with MiniOzoneCluster.
+ */
+public class TestOMSnapshotDAG {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOMSnapshotDAG.class);
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+  private static ObjectStore store;
+  private final File metaDir = OMStorage.getOmDbDir(conf);
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting OZONE_ENABLED = true
+   *
+   */
+  @BeforeAll
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    DatanodeRatisServerConfig ratisServerConfig =
+        conf.getObject(DatanodeRatisServerConfig.class);
+    ratisServerConfig.setRequestTimeOut(Duration.ofSeconds(3));
+    ratisServerConfig.setWatchTimeOut(Duration.ofSeconds(3));
+    conf.setFromObject(ratisServerConfig);
+
+    RatisClientConfig.RaftConfig raftClientConfig =
+        conf.getObject(RatisClientConfig.RaftConfig.class);
+    raftClientConfig.setRpcRequestTimeout(Duration.ofSeconds(3));
+    raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(3));
+    conf.setFromObject(raftClientConfig);
+
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(5).build();
+    cluster.waitForClusterToBeReady();
+
+    store = cluster.getClient().getObjectStore();
+
+    GenericTestUtils.setLogLevel(RaftLog.LOG, Level.INFO);
+    GenericTestUtils.setLogLevel(RaftServer.LOG, Level.INFO);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterAll
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private String getDBCheckpointAbsolutePath(SnapshotInfo snapshotInfo) {
+    return metaDir + OM_KEY_PREFIX +
+        OM_SNAPSHOT_DIR + OM_KEY_PREFIX +
+        OM_DB_NAME + snapshotInfo.getCheckpointDirName();
+  }
+
+  private static String getSnapshotDBKey(String volumeName, String bucketName,
+      String snapshotName) {
+
+    final String dbKeyPrefix = OM_KEY_PREFIX + volumeName +
+        OM_KEY_PREFIX + bucketName;
+    return dbKeyPrefix + OM_KEY_PREFIX + snapshotName;
+  }
+
+  private DifferSnapshotInfo getDifferSnapshotInfo(
+      OMMetadataManager omMetadataManager, String volumeName, String bucketName,
+      String snapshotName) throws IOException {
+
+    final String dbKey = getSnapshotDBKey(volumeName, bucketName, snapshotName);
+    final SnapshotInfo snapshotInfo =
+        omMetadataManager.getSnapshotInfoTable().get(dbKey);
+    String checkpointPath = getDBCheckpointAbsolutePath(snapshotInfo);
+
+    // Use RocksDB transaction sequence number in SnapshotInfo, which is
+    // persisted at the time of snapshot creation, as the snapshot generation
+    return new DifferSnapshotInfo(checkpointPath, snapshotInfo.getSnapshotID(),
+        snapshotInfo.getDbTxSequenceNumber());
+  }
+
+  @Test
+  void testZeroSizeKey()
+      throws IOException, InterruptedException, TimeoutException {
+
+    RandomKeyGenerator randomKeyGenerator =
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "600",
+        "--num-of-threads", "1",
+        "--key-size", "0",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
+    Assertions.assertEquals(600L, randomKeyGenerator.getNumberOfKeysAdded());
+    Assertions.assertEquals(600L,
+        randomKeyGenerator.getSuccessfulValidationCount());
+
+    List<OmVolumeArgs> volList = cluster.getOzoneManager()
+        .listAllVolumes("", "", 10);
+    LOG.debug("List of all volumes: {}", volList);
+    final String volumeName = volList.stream().filter(e ->
+        !e.getVolume().equals(OZONE_S3_VOLUME_NAME_DEFAULT))  // Ignore s3v vol
+        .collect(Collectors.toList()).get(0).getVolume();
+    List<OmBucketInfo> bucketList =
+        cluster.getOzoneManager().listBuckets(volumeName, "", "", 10);
+    LOG.debug("List of all buckets under the first volume: {}", bucketList);
+    final String bucketName = bucketList.get(0).getBucketName();
+
+    // Create snapshot
+    String resp = store.createSnapshot(volumeName, bucketName, "snap1");
+    LOG.debug("Snapshot created: {}", resp);
+
+    final OzoneVolume volume = store.getVolume(volumeName);
+    final OzoneBucket bucket = volume.getBucket(bucketName);
+
+    for (int i = 0; i < 6000; i++) {
+      bucket.createKey("b_" + i, 0).close();
+    }
+
+    // Create another snapshot
+    resp = store.createSnapshot(volumeName, bucketName, "snap3");
+    LOG.debug("Snapshot created: {}", resp);
+
+    // Get snapshot SST diff list
+    OzoneManager ozoneManager = cluster.getOzoneManager();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    RDBStore rdbStore = (RDBStore) omMetadataManager.getStore();
+    RocksDBCheckpointDiffer differ = rdbStore.getRocksDBCheckpointDiffer();
+
+    DifferSnapshotInfo snap1 = getDifferSnapshotInfo(omMetadataManager,
+        volumeName, bucketName, "snap1");
+    DifferSnapshotInfo snap3 = getDifferSnapshotInfo(omMetadataManager,
+        volumeName, bucketName, "snap3");
+
+    // RocksDB does checkpointing in a separate thread, wait for it
+    final File checkpointSnap1 = new File(snap1.getDbPath());
+    GenericTestUtils.waitFor(checkpointSnap1::exists, 2000, 20000);
+    final File checkpointSnap3 = new File(snap3.getDbPath());
+    GenericTestUtils.waitFor(checkpointSnap3::exists, 2000, 20000);
+
+    List<String> actualDiffList = differ.getSSTDiffList(snap3, snap1);
+    LOG.debug("Got diff list: {}", actualDiffList);
+    final List<String> expectedDiffList = Arrays.asList("000059");

Review Comment:
   Note this integration test is **not** complete. And will be expanded in a follow-up PR.
   
   The result is deterministic. I simply ran the test to get it. Theoretically I could do a whole RDB checkpoint open and manual SST read to find the files that holds the newly written keys in keyTable CF, but that would be reimplementing a lot of the existing differ logic in the integration test. Maybe I will do that in the follow-up jira.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012309412


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.freon;
+
+import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
+import org.apache.hadoop.hdds.utils.db.RDBStore;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMStorage;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.DifferSnapshotInfo;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.raftlog.RaftLog;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_S3_VOLUME_NAME_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+
+/**
+ * Tests Freon, with MiniOzoneCluster.
+ */
+public class TestOMSnapshotDAG {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOMSnapshotDAG.class);
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+  private static ObjectStore store;
+  private final File metaDir = OMStorage.getOmDbDir(conf);
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting OZONE_ENABLED = true
+   *
+   */
+  @BeforeAll
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    DatanodeRatisServerConfig ratisServerConfig =
+        conf.getObject(DatanodeRatisServerConfig.class);
+    ratisServerConfig.setRequestTimeOut(Duration.ofSeconds(3));
+    ratisServerConfig.setWatchTimeOut(Duration.ofSeconds(3));
+    conf.setFromObject(ratisServerConfig);
+
+    RatisClientConfig.RaftConfig raftClientConfig =
+        conf.getObject(RatisClientConfig.RaftConfig.class);
+    raftClientConfig.setRpcRequestTimeout(Duration.ofSeconds(3));
+    raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(3));
+    conf.setFromObject(raftClientConfig);
+
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(5).build();
+    cluster.waitForClusterToBeReady();
+
+    store = cluster.getClient().getObjectStore();
+
+    GenericTestUtils.setLogLevel(RaftLog.LOG, Level.INFO);
+    GenericTestUtils.setLogLevel(RaftServer.LOG, Level.INFO);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterAll
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private String getDBCheckpointAbsolutePath(SnapshotInfo snapshotInfo) {
+    return metaDir + OM_KEY_PREFIX +
+        OM_SNAPSHOT_DIR + OM_KEY_PREFIX +
+        OM_DB_NAME + snapshotInfo.getCheckpointDirName();
+  }
+
+  private static String getSnapshotDBKey(String volumeName, String bucketName,
+      String snapshotName) {
+
+    final String dbKeyPrefix = OM_KEY_PREFIX + volumeName +
+        OM_KEY_PREFIX + bucketName;
+    return dbKeyPrefix + OM_KEY_PREFIX + snapshotName;
+  }
+
+  private DifferSnapshotInfo getDifferSnapshotInfo(
+      OMMetadataManager omMetadataManager, String volumeName, String bucketName,
+      String snapshotName) throws IOException {
+
+    final String dbKey = getSnapshotDBKey(volumeName, bucketName, snapshotName);
+    final SnapshotInfo snapshotInfo =
+        omMetadataManager.getSnapshotInfoTable().get(dbKey);
+    String checkpointPath = getDBCheckpointAbsolutePath(snapshotInfo);
+
+    // Use RocksDB transaction sequence number in SnapshotInfo, which is
+    // persisted at the time of snapshot creation, as the snapshot generation
+    return new DifferSnapshotInfo(checkpointPath, snapshotInfo.getSnapshotID(),
+        snapshotInfo.getDbTxSequenceNumber());
+  }
+
+  @Test
+  void testZeroSizeKey()
+      throws IOException, InterruptedException, TimeoutException {
+
+    RandomKeyGenerator randomKeyGenerator =
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "600",
+        "--num-of-threads", "1",
+        "--key-size", "0",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
+    Assertions.assertEquals(600L, randomKeyGenerator.getNumberOfKeysAdded());
+    Assertions.assertEquals(600L,
+        randomKeyGenerator.getSuccessfulValidationCount());
+
+    List<OmVolumeArgs> volList = cluster.getOzoneManager()
+        .listAllVolumes("", "", 10);
+    LOG.debug("List of all volumes: {}", volList);
+    final String volumeName = volList.stream().filter(e ->
+        !e.getVolume().equals(OZONE_S3_VOLUME_NAME_DEFAULT))  // Ignore s3v vol
+        .collect(Collectors.toList()).get(0).getVolume();
+    List<OmBucketInfo> bucketList =
+        cluster.getOzoneManager().listBuckets(volumeName, "", "", 10);
+    LOG.debug("List of all buckets under the first volume: {}", bucketList);
+    final String bucketName = bucketList.get(0).getBucketName();
+
+    // Create snapshot
+    String resp = store.createSnapshot(volumeName, bucketName, "snap1");
+    LOG.debug("Snapshot created: {}", resp);
+
+    final OzoneVolume volume = store.getVolume(volumeName);
+    final OzoneBucket bucket = volume.getBucket(bucketName);
+
+    for (int i = 0; i < 6000; i++) {
+      bucket.createKey("b_" + i, 0).close();
+    }
+
+    // Create another snapshot
+    resp = store.createSnapshot(volumeName, bucketName, "snap3");
+    LOG.debug("Snapshot created: {}", resp);
+
+    // Get snapshot SST diff list
+    OzoneManager ozoneManager = cluster.getOzoneManager();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    RDBStore rdbStore = (RDBStore) omMetadataManager.getStore();
+    RocksDBCheckpointDiffer differ = rdbStore.getRocksDBCheckpointDiffer();
+
+    DifferSnapshotInfo snap1 = getDifferSnapshotInfo(omMetadataManager,
+        volumeName, bucketName, "snap1");
+    DifferSnapshotInfo snap3 = getDifferSnapshotInfo(omMetadataManager,
+        volumeName, bucketName, "snap3");
+
+    // RocksDB does checkpointing in a separate thread, wait for it
+    final File checkpointSnap1 = new File(snap1.getDbPath());
+    GenericTestUtils.waitFor(checkpointSnap1::exists, 2000, 20000);
+    final File checkpointSnap3 = new File(snap3.getDbPath());
+    GenericTestUtils.waitFor(checkpointSnap3::exists, 2000, 20000);
+
+    List<String> actualDiffList = differ.getSSTDiffList(snap3, snap1);
+    LOG.debug("Got diff list: {}", actualDiffList);
+    final List<String> expectedDiffList = Arrays.asList("000059");

Review Comment:
   Note this integration test is **not** complete. And will be expanded in a follow-up PR.
   
   The result is deterministic. I simply ran the test to get it. Theoretically I could do a whole RDB checkpoint open and manual SST read to find the files that holds the keyTable CF but that would be reimplementing a lot of the existing differ logic. Maybe I will do that in the follow-up jira.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1000031922


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -131,6 +132,22 @@ public static DBCheckpoint createOmSnapshotCheckpoint(
       OMMetadataManager omMetadataManager, SnapshotInfo snapshotInfo)
       throws IOException {
     RDBStore store = (RDBStore) omMetadataManager.getStore();
+
+    final long dbLatestSequenceNumber =
+        ((RDBStore) omMetadataManager.getStore()).getDb()
+            .getLatestSequenceNumber();
+
+    final RocksDBCheckpointDiffer checkpointDiffer =
+        omMetadataManager.getStore().getRocksDBCheckpointDiffer();
+
+    // Set compaction log filename to the latest DB sequence number
+    // right before taking the RocksDB checkpoint
+    //
+    // Note it doesn't matter if sequence number hasn't increased (even though
+    // it shouldn't happen), since the writer always appends the file.
+    checkpointDiffer.setCompactionLogParentDir(store.getSnapshotsParentDir());
+    checkpointDiffer.setCompactionLogFilenameBySeqNum(dbLatestSequenceNumber);
+

Review Comment:
   In the latest commit e76a24eddefcc44ab429f2be3245e0ec252c7d5a I have added the logic to persist sequence number at the time of checkpoint (snapshot) creationm, so that it can be read during DAG reconstruction.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -131,6 +132,22 @@ public static DBCheckpoint createOmSnapshotCheckpoint(
       OMMetadataManager omMetadataManager, SnapshotInfo snapshotInfo)
       throws IOException {
     RDBStore store = (RDBStore) omMetadataManager.getStore();
+
+    final long dbLatestSequenceNumber =
+        ((RDBStore) omMetadataManager.getStore()).getDb()
+            .getLatestSequenceNumber();
+
+    final RocksDBCheckpointDiffer checkpointDiffer =
+        omMetadataManager.getStore().getRocksDBCheckpointDiffer();
+
+    // Set compaction log filename to the latest DB sequence number
+    // right before taking the RocksDB checkpoint
+    //
+    // Note it doesn't matter if sequence number hasn't increased (even though
+    // it shouldn't happen), since the writer always appends the file.
+    checkpointDiffer.setCompactionLogParentDir(store.getSnapshotsParentDir());
+    checkpointDiffer.setCompactionLogFilenameBySeqNum(dbLatestSequenceNumber);
+

Review Comment:
   In the latest commit e76a24eddefcc44ab429f2be3245e0ec252c7d5a I have added the logic to persist sequence number at the time of snapshot creation, so that it can be read during DAG reconstruction.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1000011978


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -230,196 +330,154 @@ public void createCheckPoint(String dbPathArg, String cpPathArg,
     }
   }
 
+  /**
+   * Append (then flush) to the current compaction log file path.
+   */
+  public void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogFilename == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Current compaction log filename is not set. "
+          + "Please check initialization.");
+      return;
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogFilename),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogFilename, e);
+    }
+  }
+
+  /**
+   * This takes Options.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  /**
+   * This takes DBOptions.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
   public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (db) {

Review Comment:
   You have a point.
   
   @prashantpogde a few weeks back explained to me the reason behind having to synchronize `db` here, I was convinced at the time, but I can't recall it any more. Possibly the logic has been changed enough that it is no longer necessary. In that case I would drop it.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1011376445


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -744,17 +744,18 @@ enum SnapshotStatusProto {
  * SnapshotInfo table entry for Bucket/Volume snapshot metadata
  */
 message SnapshotInfo {
-  required string snapshotID = 1;
-  required string name = 2;
-  required string volumeName = 3;
-  required string bucketName = 4;
-  required SnapshotStatusProto snapshotStatus = 5;
-  required uint64 creationTime = 6;
-  required uint64 deletionTime = 7;
-  required string pathPreviousSnapshotID = 8;
-  required string globalPreviousSnapshotID = 9;
-  required string snapshotPath = 10;
-  required string checkpointDir = 11;
+  optional string snapshotID = 1;
+  optional string name = 2;
+  optional string volumeName = 3;
+  optional string bucketName = 4;
+  optional SnapshotStatusProto snapshotStatus = 5;
+  optional uint64 creationTime = 6;
+  optional uint64 deletionTime = 7;
+  optional string pathPreviousSnapshotID = 8;
+  optional string globalPreviousSnapshotID = 9;
+  optional string snapshotPath = 10;
+  optional string checkpointDir = 11;
+  optional int64 dbTxSequenceNumber = 12;

Review Comment:
   To whoever reviewing this: the rationale behind **always** using `optional` protobuf fields from now on is that proto3 has already ditched `required`, meaning **all** proto3 fields are `optional`.
   
   More reading: https://stackoverflow.com/a/52360213
   
   Though we are still using proto2 in this case, using `optional` should make the future conversion to proto3 easier (if that day ever comes).
   
   Also, message `SnapshotInfo` is newly added in the snapshot dev branch and not merged to master branch yet. So no compatiblity concerns here.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl merged pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl merged PR #3824:
URL: https://github.com/apache/ozone/pull/3824


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] prashantpogde commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1004935420


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java:
##########
@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.freon;
+
+import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.raftlog.RaftLog;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_S3_VOLUME_NAME_DEFAULT;
+
+/**
+ * Tests Freon, with MiniOzoneCluster.
+ */
+public class TestOMSnapshotDAG {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOMSnapshotDAG.class);
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+  private static ObjectStore store;
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting OZONE_ENABLED = true
+   *
+   */
+  @BeforeAll
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    DatanodeRatisServerConfig ratisServerConfig =
+        conf.getObject(DatanodeRatisServerConfig.class);
+    ratisServerConfig.setRequestTimeOut(Duration.ofSeconds(3));
+    ratisServerConfig.setWatchTimeOut(Duration.ofSeconds(3));
+    conf.setFromObject(ratisServerConfig);
+
+    RatisClientConfig.RaftConfig raftClientConfig =
+        conf.getObject(RatisClientConfig.RaftConfig.class);
+    raftClientConfig.setRpcRequestTimeout(Duration.ofSeconds(3));
+    raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(3));
+    conf.setFromObject(raftClientConfig);
+
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(5).build();
+    cluster.waitForClusterToBeReady();
+
+    store = cluster.getClient().getObjectStore();
+
+    GenericTestUtils.setLogLevel(RaftLog.LOG, Level.INFO);
+    GenericTestUtils.setLogLevel(RaftServer.LOG, Level.INFO);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterAll
+  public static void shutdown() {
+    if (cluster != null) {
+      LOG.warn("Waiting for an extra 10 seconds before shutting down...");
+      try {
+        Thread.sleep(10000);
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  void testZeroSizeKey() throws IOException {
+
+    RandomKeyGenerator randomKeyGenerator =
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "6000",
+        "--num-of-threads", "1",
+        "--key-size", "0",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
+    Assert.assertEquals(6000L, randomKeyGenerator.getNumberOfKeysAdded());
+    Assert.assertEquals(6000L,
+        randomKeyGenerator.getSuccessfulValidationCount());
+
+    List<OmVolumeArgs> volList = cluster.getOzoneManager()
+        .listAllVolumes("", "", 10);
+    System.out.println(volList);
+    final String volumeName = volList.stream().filter(e ->
+        !e.getVolume().equals(OZONE_S3_VOLUME_NAME_DEFAULT))  // Ignore s3v vol
+        .collect(Collectors.toList()).get(0).getVolume();
+    List<OmBucketInfo> bucketList =
+        cluster.getOzoneManager().listBuckets(volumeName, "", "", 10);
+    System.out.println(bucketList);
+    final String bucketName = bucketList.get(0).getBucketName();
+
+    // Create snapshot
+    String resp = store.createSnapshot(volumeName, bucketName, "snap1");
+    System.out.println(resp);
+
+    final OzoneVolume volume = store.getVolume(volumeName);
+    final OzoneBucket bucket = volume.getBucket(bucketName);
+
+    for (int i = 0; i < 6000; i++) {
+      bucket.createKey("b_" + i, 0).close();
+    }
+
+    resp = store.createSnapshot(volumeName, bucketName, "snap3");
+    System.out.println(resp);
+  }
+

Review Comment:
   We are not really testing the DAG and DAG persistence here.
   We could do several things e.g. : 
   1) Create Snap1 & Snap2 and check that the DAG correctly returns the diff file.
   2) We can also check the DAG persistent part and validating that the  DAG reconstructed after system restart returns the same result as 1) above.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] prashantpogde commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1004945912


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,38 +539,213 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  private String[] sstTokensRead;
+  private long reconstructionSnapshotGeneration;
+
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   */
+  private synchronized void processCompactionLogLine(String line) {
+
+    LOG.debug("Processing line: {}", line);
+
+    // Skip comments
+    if (line.startsWith("#")) {
+      LOG.debug("Comment line, skipped");
+      return;
+    }
+
+    // Read sequence number as the snapshot generation for the nodes
+    if (line.startsWith(COMPACTION_LOG_SEQNUM_LINE_PREFIX)) {
+      LOG.debug("Reading sequence number as snapshot generation");
+      final String seqNumStr =
+          line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length()).trim();
+      reconstructionSnapshotGeneration = Long.parseLong(seqNumStr);
+      return;
+    }
+
+    if (sstTokensRead == null) {
+      // Store the tokens in the first line
+      sstTokensRead = line.split("\t");
+      LOG.debug("Number of input files: {}", sstTokensRead.length);
+      if (sstTokensRead.length == 0) {
+        // Sanity check. inputFiles should never be empty. outputFiles could.
+        throw new RuntimeException(
+            "Compaction inputFiles list is empty. File is corrupted?");
+      }
+    } else {
+      final String[] outputFilesRead = line.split("\t");
+      LOG.debug("Number of output files: {}", outputFilesRead.length);
+
+      // Populate the compaction DAG
+      populateCompactionDAG(sstTokensRead, outputFilesRead,
+          reconstructionSnapshotGeneration);
+
+      // Reset inputFilesRead to null so
+      sstTokensRead = null;
+    }
+  }
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   * <p>
+   * DO NOT use this function in another context without understanding what it
+   * does, due to the state preserved between calls (in sstTokensRead).
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {

Review Comment:
   We should also trigger it on OM restart



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] DaveTeng0 commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
DaveTeng0 commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012316485


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -744,17 +744,18 @@ enum SnapshotStatusProto {
  * SnapshotInfo table entry for Bucket/Volume snapshot metadata
  */
 message SnapshotInfo {
-  required string snapshotID = 1;
-  required string name = 2;
-  required string volumeName = 3;
-  required string bucketName = 4;
-  required SnapshotStatusProto snapshotStatus = 5;
-  required uint64 creationTime = 6;
-  required uint64 deletionTime = 7;
-  required string pathPreviousSnapshotID = 8;
-  required string globalPreviousSnapshotID = 9;
-  required string snapshotPath = 10;
-  required string checkpointDir = 11;
+  optional string snapshotID = 1;
+  optional string name = 2;
+  optional string volumeName = 3;
+  optional string bucketName = 4;
+  optional SnapshotStatusProto snapshotStatus = 5;
+  optional uint64 creationTime = 6;
+  optional uint64 deletionTime = 7;
+  optional string pathPreviousSnapshotID = 8;
+  optional string globalPreviousSnapshotID = 9;
+  optional string snapshotPath = 10;
+  optional string checkpointDir = 11;
+  optional int64 dbTxSequenceNumber = 12;

Review Comment:
   thanks siyao for the reminder!



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1011103765


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,134 +558,231 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   */
+  private synchronized void processCompactionLogLine(String line) {
+
+    LOG.debug("Processing line: {}", line);
+
+    if (line.startsWith("#")) {
+      // Skip comments
+      LOG.debug("Comment line, skipped");
+    } else if (line.startsWith(COMPACTION_LOG_SEQNUM_LINE_PREFIX)) {
+      // Read sequence number
+      LOG.debug("Reading sequence number as snapshot generation");
+      final String seqNumStr =
+          line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length()).trim();
+      // This would the snapshot generation for the nodes to come
+      reconstructionSnapshotGeneration = Long.parseLong(seqNumStr);
+    } else if (line.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) {
+      // Read compaction log entry
+
+      // Trim the beginning
+      line = line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length());
+      final String[] io = line.split(":");
+      if (io.length != 2) {
+        LOG.error("Invalid line in compaction log: {}", line);
+        return;
+      }
+      final String[] inputFiles = io[0].split(",");
+      final String[] outputFiles = io[1].split(",");
+      populateCompactionDAG(Arrays.asList(inputFiles),
+          Arrays.asList(outputFiles), reconstructionSnapshotGeneration);
+    } else {
+      LOG.error("Invalid line in compaction log: {}", line);
+    }
+  }
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> logLineStream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      logLineStream.forEach(this::processCompactionLogLine);
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Load existing compaction log files to the in-memory DAG.
+   * This only needs to be done once during OM startup.
+   */
+  public synchronized void loadAllCompactionLogs() {
+    if (compactionLogDir == null) {
+      throw new RuntimeException("Compaction log directory must be set first");
+    }
+    reconstructionSnapshotGeneration = 0L;
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase()
+              .endsWith(COMPACTION_LOG_FILENAME_SUFFIX))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+          readCompactionLogToDAG(logPath.toString());
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+  }
+
+  /**
+   * Snapshot information node class in the DAG.
+   */
+  static class Snapshot {
+    private final String dbPath;
+    private final String snapshotID;
+    private final long snapshotGeneration;
+
+    Snapshot(String db, String id, long gen) {
+      dbPath = db;
+      snapshotID = id;
+      snapshotGeneration = gen;
+    }
+
+    public String getDbPath() {
+      return dbPath;
+    }
+
+    public String getSnapshotID() {
+      return snapshotID;
+    }
+
+    public long getSnapshotGeneration() {
+      return snapshotGeneration;
+    }
+  }
+
+  /**
+   * Get a list of SST files that differs between src and destination snapshots.
+   * <p>
+   * Expected input: src is a snapshot taken AFTER the dest.
+   *
+   * @param src source snapshot
+   * @param dest destination snapshot
+   */
+  public synchronized List<String> getSSTDiffList(Snapshot src, Snapshot dest) {
+
+    LOG.debug("src '{}' -> dest '{}'", src.dbPath, dest.dbPath);
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
 
     HashSet<String> fwdDAGSameFiles = new HashSet<>();
     HashSet<String> fwdDAGDifferentFiles = new HashSet<>();
 
-    LOG.warn("Doing forward diff between source and destination " +
-        "Snapshots:" + src.dbPath + ", " + dest.dbPath);
-    realPrintSnapdiffSSTFiles(src, dest, srcSnapFiles, destSnapFiles,
-        compactionDAGFwd,
-        fwdDAGSameFiles,
-        fwdDAGDifferentFiles);
+    LOG.debug("Doing forward diff between src and dest snapshots: " +
+        src.dbPath + " to " + dest.dbPath);
+    internalGetSSTDiffList(src, dest, srcSnapFiles, destSnapFiles,
+        compactionDAGFwd, fwdDAGSameFiles, fwdDAGDifferentFiles);
 
-    LOG.warn("Overall Summary \n" +
-            "Doing Overall diff between source and destination Snapshots:" +
-        src.dbPath + ", " + dest.dbPath);
-    System.out.print("fwd DAG Same files :");
-    for (String file : fwdDAGSameFiles) {
-      System.out.print(file + ", ");
-    }
-    LOG.warn("");
-    System.out.print("\nFwd DAG Different files :");
-    for (String file : fwdDAGDifferentFiles) {
-      CompactionNode n = compactionNodeTable.get(file);
-      System.out.print(file + ", ");
+    List<String> res = new ArrayList<>();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Result of diff from src '" + src.dbPath + "' to dest '" +
+          dest.dbPath + "':");
+      StringBuilder logSB = new StringBuilder();
+
+      logSB.append("Fwd DAG same SST files:      ");
+      for (String file : fwdDAGSameFiles) {
+        logSB.append(file).append(" ");
+      }
+      LOG.debug(logSB.toString());
+
+      logSB.setLength(0);
+      logSB.append("Fwd DAG different SST files: ");
+      for (String file : fwdDAGDifferentFiles) {
+        logSB.append(file).append(" ");
+        res.add(file);
+      }
+      LOG.debug(logSB.toString());
+
+    } else {
+      res.addAll(fwdDAGDifferentFiles);
     }
-    LOG.warn("");
+
+    return res;
   }
 
+  /**
+   * Core getSSTDiffList logic.
+   */
   @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-  public synchronized void realPrintSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest,
-      HashSet<String> srcSnapFiles,
-      HashSet<String> destSnapFiles,
+  private void internalGetSSTDiffList(Snapshot src, Snapshot dest,
+      HashSet<String> srcSnapFiles, HashSet<String> destSnapFiles,
       MutableGraph<CompactionNode> mutableGraph,
       HashSet<String> sameFiles, HashSet<String> differentFiles) {
 
-
     for (String fileName : srcSnapFiles) {
       if (destSnapFiles.contains(fileName)) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + " and Dest " +
-            "Snapshot" + dest.dbPath + " Contain Same file " + fileName);
+        LOG.debug("Source '{}' and destination '{}' share the same SST '{}'",
+            src.dbPath, dest.dbPath, fileName);
         sameFiles.add(fileName);
         continue;
       }
       CompactionNode infileNode =
           compactionNodeTable.get(Paths.get(fileName).getFileName().toString());
       if (infileNode == null) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + "File " + fileName + "was " +
-            "never compacted");
+        LOG.debug("Src " + src.dbPath + " File " + fileName +
+            " was never compacted");
         differentFiles.add(fileName);
         continue;
       }
-      System.out.print(" Expandin File:" + fileName + ":\n");
-      Set<CompactionNode> nextLevel = new HashSet<>();
-      nextLevel.add(infileNode);
+      LOG.debug("Expanding SST file: " + fileName);
       Set<CompactionNode> currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
-      nextLevel = new HashSet<>();
+      currentLevel.add(infileNode);
+      Set<CompactionNode> nextLevel = new HashSet<>();
       int i = 1;
       while (currentLevel.size() != 0) {

Review Comment:
   done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1009990346


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -131,7 +132,30 @@ public static DBCheckpoint createOmSnapshotCheckpoint(
       OMMetadataManager omMetadataManager, SnapshotInfo snapshotInfo)
       throws IOException {
     RDBStore store = (RDBStore) omMetadataManager.getStore();
-    return store.getSnapshot(snapshotInfo.getCheckpointDirName());
+
+    final long dbLatestSequenceNumber =
+        ((RDBStore) omMetadataManager.getStore()).getDb()
+            .getLatestSequenceNumber();
+
+    final RocksDBCheckpointDiffer dbCpDiffer =
+        omMetadataManager.getStore().getRocksDBCheckpointDiffer();
+
+    final DBCheckpoint dbCheckpoint = store.getSnapshot(
+        snapshotInfo.getCheckpointDirName());
+
+    // Write snapshot generation (latest sequence number) to compaction log.
+    // This will be used for DAG reconstruction as snapshotGeneration.
+    dbCpDiffer.appendSequenceNumberToCompactionLog(dbLatestSequenceNumber);
+
+    // Set compaction log filename to the latest DB sequence number
+    // right after taking the RocksDB checkpoint for Ozone snapshot.
+    //
+    // Note it doesn't matter if sequence number hasn't increased (even though
+    // it shouldn't happen), since the writer always appends the file.
+    dbCpDiffer.setCompactionLogParentDir(store.getSnapshotsParentDir());

Review Comment:
   Does the parent dir get set every time a snapshot is created?



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -131,7 +132,30 @@ public static DBCheckpoint createOmSnapshotCheckpoint(
       OMMetadataManager omMetadataManager, SnapshotInfo snapshotInfo)
       throws IOException {
     RDBStore store = (RDBStore) omMetadataManager.getStore();
-    return store.getSnapshot(snapshotInfo.getCheckpointDirName());
+
+    final long dbLatestSequenceNumber =
+        ((RDBStore) omMetadataManager.getStore()).getDb()
+            .getLatestSequenceNumber();
+
+    final RocksDBCheckpointDiffer dbCpDiffer =
+        omMetadataManager.getStore().getRocksDBCheckpointDiffer();
+
+    final DBCheckpoint dbCheckpoint = store.getSnapshot(
+        snapshotInfo.getCheckpointDirName());
+
+    // Write snapshot generation (latest sequence number) to compaction log.
+    // This will be used for DAG reconstruction as snapshotGeneration.
+    dbCpDiffer.appendSequenceNumberToCompactionLog(dbLatestSequenceNumber);
+
+    // Set compaction log filename to the latest DB sequence number
+    // right after taking the RocksDB checkpoint for Ozone snapshot.
+    //
+    // Note it doesn't matter if sequence number hasn't increased (even though
+    // it shouldn't happen), since the writer always appends the file.
+    dbCpDiffer.setCompactionLogParentDir(store.getSnapshotsParentDir());

Review Comment:
   Does the parent dir need to get set every time a snapshot is created?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] prashantpogde commented on pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on PR #3824:
URL: https://github.com/apache/ozone/pull/3824#issuecomment-1291170705

   Current  compaction log format is
   input files newline output files
   
   Its a very risky format. Lets change it to
   something like
   
   C(compaction log entry) $ input-file1, i2, i3 : outputfile-1, o2 o3                                                       
   S(Snapshot creation entry) $ Snapshot number                                                           
   C(compaction log entry) $ input-file1, i2, i3 : outputfile-1, o2 o3
   
   But this can be done in a separate PR also and a separate Jira. Lets get this one in as it is working. All improvements can happen incrementally.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993957587


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -230,196 +330,154 @@ public void createCheckPoint(String dbPathArg, String cpPathArg,
     }
   }
 
+  /**
+   * Append (then flush) to the current compaction log file path.
+   */
+  public void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogFilename == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Current compaction log filename is not set. "
+          + "Please check initialization.");
+      return;
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogFilename),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogFilename, e);
+    }
+  }
+
+  /**
+   * This takes Options.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  /**
+   * This takes DBOptions.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
   public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (db) {

Review Comment:
   I think it is a bit dangerous to synchronize on an object we don't have control over.  I'm not sure what else is synchronizing against it.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993951315


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -114,9 +113,82 @@ public class RocksDBCheckpointDiffer {
   private static final long UNKNOWN_COMPACTION_GEN = 0;
   private long currentCompactionGen = 0;
 
-  // Something to track all the snapshots created so far.
+  // Something to track all the snapshots created so far. TODO: REMOVE
   private Snapshot[] allSnapshots;
 
+  private String compactionLogParentDir = null;
+  private String compactionLogDir = null;
+
+  // Name of the directory to hold compaction logs (under parent dir)
+  private static final String COMPACTION_LOG_DIR = "compaction-log/";
+
+  // For DB compaction SST DAG persistence and reconstruction
+  // Should be initialized to the latest sequence number
+  private volatile String currentCompactionLogFilename = null;
+
+  private static final String COMPACTION_LOG_FILENAME_SUFFIX = ".log";
+
+  public void setCompactionLogParentDir(String parentDir) {
+    this.compactionLogParentDir = parentDir;
+
+    // Append /
+    if (!compactionLogParentDir.endsWith("/")) {
+      compactionLogParentDir += "/";
+    }
+
+    File pDir = new File(compactionLogParentDir);
+    if (!pDir.exists()) {
+      if (!pDir.mkdir()) {
+        LOG.error("Error creating compaction log parent dir.");
+        return;
+      }
+    }
+
+    compactionLogDir =
+        Paths.get(compactionLogParentDir, COMPACTION_LOG_DIR).toString();
+    File clDir = new File(compactionLogDir);
+    if (!clDir.mkdir()) {
+      LOG.error("Error creating compaction log dir.");
+      return;
+    }
+
+    // TODO: Write a README there explaining what the dir is for
+  }
+
+  private static final int LONG_MAX_STRLEN =
+      String.valueOf(Long.MAX_VALUE).length();
+
+  public void setCompactionLogFilenameBySeqNum(long latestSequenceId) {

Review Comment:
   so as to match the other name: "appendToCurrentCompactionLog()"?
   



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1000032300


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -145,16 +235,29 @@ public RocksDBCheckpointDiffer(String dbPath,
     //  mapping.
   }
 
+  /**
+   * Helper function that recursively deletes the dir. TODO: REMOVE
+   */
+  boolean deleteDirectory(File directoryToBeDeleted) {
+    File[] allContents = directoryToBeDeleted.listFiles();
+    if (allContents != null) {
+      for (File file : allContents) {
+        deleteDirectory(file);
+      }
+    }
+    return directoryToBeDeleted.delete();
+  }
+
   // Node in the DAG to represent an SST file
   private class CompactionNode {
-    public String fileName;   // Name of the SST file
-    public String snapshotId; // The last snapshot that was created before this
+    private String fileName;   // Name of the SST file
+    private String snapshotId; // The last snapshot that was created before this
     // node came into existance;
-    public long snapshotGeneration;
-    public long totalNumberOfKeys;
-    public long cumulativeKeysReverseTraversal;
+    private long snapshotGeneration;
+    private long totalNumberOfKeys;
+    private long cumulativeKeysReverseTraversal;
 
-    CompactionNode (String f, String sid, long numKeys, long compactionGen) {
+    CompactionNode(String f, String sid, long numKeys, long compactionGen) {

Review Comment:
   Yes. In the latest commit it has been repurposed to pass in the sequence number as snapshot generation.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1000011978


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -230,196 +330,154 @@ public void createCheckPoint(String dbPathArg, String cpPathArg,
     }
   }
 
+  /**
+   * Append (then flush) to the current compaction log file path.
+   */
+  public void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogFilename == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Current compaction log filename is not set. "
+          + "Please check initialization.");
+      return;
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogFilename),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogFilename, e);
+    }
+  }
+
+  /**
+   * This takes Options.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  /**
+   * This takes DBOptions.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
   public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (db) {

Review Comment:
   You have a point.
   
   @prashantpogde a few weeks back explained to me the reason behind having to synchronize `db` here. I was convinced at the time. But I can't recall anymore. Possibly the logic has been changed enough that it is no longer necessary. In that case I would drop it.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] hemantk-12 commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1011056164


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -688,130 +868,139 @@ public synchronized void printMutableGraph(
       iter.remove();
       iter = topLevelNodes.iterator();
     }
-    LOG.warn("src snap:" + srcSnapId);
-    LOG.warn("dest snap:" + destSnapId);
+    LOG.debug("src snap: " + srcSnapId);
+    LOG.debug("dest snap: " + destSnapId);
     for (CompactionNode n : allNodes) {
-      LOG.warn("Files are :" + n.fileName);
+      LOG.debug("Files are: " + n.fileName);
     }
   }
 
+  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
+    return compactionDAGFwd;
+  }
 
-  public void createSnapshot(RocksDB rocksDB) throws InterruptedException {
-
-    LOG.warn("Current time is::" + System.currentTimeMillis());
-    long t1 = System.currentTimeMillis();
-
-    cpPath = cpPath + lastSnapshotCounter;
-    createCheckPoint(rocksDbPath, cpPath, rocksDB);
-    allSnapshots[lastSnapshotCounter] = new Snapshot(cpPath,
-    lastSnapshotPrefix, lastSnapshotCounter);
-
-    long t2 = System.currentTimeMillis();
-    LOG.warn("Current time is::" + t2);
-
-    LOG.warn("millisecond difference is ::" + (t2 - t1));
-   Thread.sleep(100);
-   ++lastSnapshotCounter;
-   lastSnapshotPrefix = "sid_" + lastSnapshotCounter;
-   LOG.warn("done :: 1");
+  public MutableGraph<CompactionNode> getCompactionReverseDAG() {
+    return compactionDAGReverse;
   }
 
+  /**
+   * Populate the compaction DAG with input and output SST files lists.
+   */
+  @SuppressFBWarnings({"AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION"})
+  private void populateCompactionDAG(List<String> inputFiles,
+      List<String> outputFiles, long seqNum) {
 
-  public void printAllSnapshots() throws InterruptedException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
-      }
-      LOG.warn("Snapshot id" + snap.snapshotID);
-      LOG.warn("Snapshot path" + snap.dbPath);
-      LOG.warn("Snapshot Generation" + snap.snapshotGeneration);
-      LOG.warn("");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Input {} -> Output {}", inputFiles, outputFiles);
     }
-  }
 
-  public void diffAllSnapshots() throws InterruptedException, RocksDBException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
+    for (String outfile : outputFiles) {
+      CompactionNode outfileNode = compactionNodeTable.get(outfile);
+      if (outfileNode == null) {
+        long numKeys = 0L;
+        try {
+          numKeys = getSSTFileSummary(outfile);
+        } catch (Exception e) {
+          LOG.warn("Exception in getSSTFileSummary: {}", e.getMessage());
+        }
+        outfileNode = new CompactionNode(outfile, null, numKeys, seqNum);
+        compactionDAGFwd.addNode(outfileNode);
+        compactionDAGReverse.addNode(outfileNode);
+        compactionNodeTable.put(outfile, outfileNode);
       }
-      printSnapdiffSSTFiles(allSnapshots[lastSnapshotCounter - 1], snap);
-    }
-  }
 
-  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
-    return compactionDAGFwd;
-  }
+      for (String infile : inputFiles) {
+        CompactionNode infileNode = compactionNodeTable.get(infile);

Review Comment:
   Code, from line 898-910 and 913-925, is same. You can extract out the common code to helper function.
   
   ```
     public CompactionNode getCompactionNode(String file) {
       CompactionNode fileNode = compactionNodeTable.get(file);
       if (fileNode != null) {
   
         long numKeys = 0L;
         try {
           numKeys = getSSTFileSummary(outfile);
         } catch (Exception e) {
           LOG.warn("Exception in getSSTFileSummary: {}", e.getMessage());
         }
   
         fileNode = new CompactionNode(file, null, numKeys, seqNum);
         compactionDAGFwd.addNode(fileNode);
         compactionDAGReverse.addNode(fileNode);
         compactionNodeTable.put(file, fileNode);
       }
       return fileNode;
     }
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012307040


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.freon;
+
+import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
+import org.apache.hadoop.hdds.utils.db.RDBStore;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMStorage;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.DifferSnapshotInfo;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.raftlog.RaftLog;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_S3_VOLUME_NAME_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+
+/**
+ * Tests Freon, with MiniOzoneCluster.
+ */
+public class TestOMSnapshotDAG {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOMSnapshotDAG.class);
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+  private static ObjectStore store;
+  private final File metaDir = OMStorage.getOmDbDir(conf);
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting OZONE_ENABLED = true
+   *
+   */
+  @BeforeAll
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    DatanodeRatisServerConfig ratisServerConfig =
+        conf.getObject(DatanodeRatisServerConfig.class);
+    ratisServerConfig.setRequestTimeOut(Duration.ofSeconds(3));
+    ratisServerConfig.setWatchTimeOut(Duration.ofSeconds(3));
+    conf.setFromObject(ratisServerConfig);
+
+    RatisClientConfig.RaftConfig raftClientConfig =
+        conf.getObject(RatisClientConfig.RaftConfig.class);
+    raftClientConfig.setRpcRequestTimeout(Duration.ofSeconds(3));
+    raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(3));
+    conf.setFromObject(raftClientConfig);
+
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(5).build();
+    cluster.waitForClusterToBeReady();
+
+    store = cluster.getClient().getObjectStore();
+
+    GenericTestUtils.setLogLevel(RaftLog.LOG, Level.INFO);
+    GenericTestUtils.setLogLevel(RaftServer.LOG, Level.INFO);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterAll
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private String getDBCheckpointAbsolutePath(SnapshotInfo snapshotInfo) {
+    return metaDir + OM_KEY_PREFIX +
+        OM_SNAPSHOT_DIR + OM_KEY_PREFIX +
+        OM_DB_NAME + snapshotInfo.getCheckpointDirName();
+  }
+
+  private static String getSnapshotDBKey(String volumeName, String bucketName,
+      String snapshotName) {
+
+    final String dbKeyPrefix = OM_KEY_PREFIX + volumeName +
+        OM_KEY_PREFIX + bucketName;
+    return dbKeyPrefix + OM_KEY_PREFIX + snapshotName;
+  }
+
+  private DifferSnapshotInfo getDifferSnapshotInfo(
+      OMMetadataManager omMetadataManager, String volumeName, String bucketName,
+      String snapshotName) throws IOException {
+
+    final String dbKey = getSnapshotDBKey(volumeName, bucketName, snapshotName);
+    final SnapshotInfo snapshotInfo =
+        omMetadataManager.getSnapshotInfoTable().get(dbKey);
+    String checkpointPath = getDBCheckpointAbsolutePath(snapshotInfo);
+
+    // Use RocksDB transaction sequence number in SnapshotInfo, which is
+    // persisted at the time of snapshot creation, as the snapshot generation
+    return new DifferSnapshotInfo(checkpointPath, snapshotInfo.getSnapshotID(),
+        snapshotInfo.getDbTxSequenceNumber());
+  }
+
+  @Test
+  void testZeroSizeKey()
+      throws IOException, InterruptedException, TimeoutException {
+
+    RandomKeyGenerator randomKeyGenerator =
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "600",
+        "--num-of-threads", "1",
+        "--key-size", "0",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
+    Assertions.assertEquals(600L, randomKeyGenerator.getNumberOfKeysAdded());
+    Assertions.assertEquals(600L,
+        randomKeyGenerator.getSuccessfulValidationCount());
+
+    List<OmVolumeArgs> volList = cluster.getOzoneManager()
+        .listAllVolumes("", "", 10);
+    LOG.debug("List of all volumes: {}", volList);
+    final String volumeName = volList.stream().filter(e ->
+        !e.getVolume().equals(OZONE_S3_VOLUME_NAME_DEFAULT))  // Ignore s3v vol
+        .collect(Collectors.toList()).get(0).getVolume();
+    List<OmBucketInfo> bucketList =
+        cluster.getOzoneManager().listBuckets(volumeName, "", "", 10);
+    LOG.debug("List of all buckets under the first volume: {}", bucketList);
+    final String bucketName = bucketList.get(0).getBucketName();
+
+    // Create snapshot
+    String resp = store.createSnapshot(volumeName, bucketName, "snap1");
+    LOG.debug("Snapshot created: {}", resp);
+
+    final OzoneVolume volume = store.getVolume(volumeName);
+    final OzoneBucket bucket = volume.getBucket(bucketName);
+
+    for (int i = 0; i < 6000; i++) {
+      bucket.createKey("b_" + i, 0).close();
+    }
+
+    // Create another snapshot
+    resp = store.createSnapshot(volumeName, bucketName, "snap3");
+    LOG.debug("Snapshot created: {}", resp);
+
+    // Get snapshot SST diff list
+    OzoneManager ozoneManager = cluster.getOzoneManager();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    RDBStore rdbStore = (RDBStore) omMetadataManager.getStore();
+    RocksDBCheckpointDiffer differ = rdbStore.getRocksDBCheckpointDiffer();
+
+    DifferSnapshotInfo snap1 = getDifferSnapshotInfo(omMetadataManager,
+        volumeName, bucketName, "snap1");
+    DifferSnapshotInfo snap3 = getDifferSnapshotInfo(omMetadataManager,
+        volumeName, bucketName, "snap3");
+
+    // RocksDB does checkpointing in a separate thread, wait for it
+    final File checkpointSnap1 = new File(snap1.getDbPath());
+    GenericTestUtils.waitFor(checkpointSnap1::exists, 2000, 20000);
+    final File checkpointSnap3 = new File(snap3.getDbPath());
+    GenericTestUtils.waitFor(checkpointSnap3::exists, 2000, 20000);
+
+    List<String> actualDiffList = differ.getSSTDiffList(snap3, snap1);
+    LOG.debug("Got diff list: {}", actualDiffList);
+    final List<String> expectedDiffList = Arrays.asList("000059");

Review Comment:
   I don't understand where "000059" comes from?  How do we know that is the filename?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012330204


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java:
##########
@@ -29,60 +31,114 @@
 import java.util.Random;
 import java.util.stream.Collectors;
 
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.DifferSnapshotInfo;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.rocksdb.Checkpoint;
 import org.rocksdb.ColumnFamilyDescriptor;
 import org.rocksdb.ColumnFamilyHandle;
 import org.rocksdb.ColumnFamilyOptions;
-import org.rocksdb.CompressionType;
 import org.rocksdb.DBOptions;
+import org.rocksdb.FlushOptions;
 import org.rocksdb.LiveFileMetaData;
 import org.rocksdb.Options;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
 import org.rocksdb.RocksIterator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
 
-////CHECKSTYLE:OFF
+/**
+ * Test RocksDBCheckpointDiffer basic functionality.
+ */
 public class TestRocksDBCheckpointDiffer {
 
-  private static final String dbPath   = "./rocksdb-data";
-  private static final int NUM_ROW = 25000000;
-  private static final int SNAPSHOT_EVERY_SO_MANY_KEYS = 999999;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRocksDBCheckpointDiffer.class);
+
+  /**
+   * RocksDB path for the test.
+   */
+  private static final String TEST_DB_PATH = "./rocksdb-data";
+  private static final int NUM_ROW = 250000;
+  private static final int SNAPSHOT_EVERY_SO_MANY_KEYS = 49999;
+
+  /**
+   * RocksDB checkpoint path prefix.
+   */
+  private static final String CP_PATH_PREFIX = "rocksdb-cp-";
+  private final ArrayList<DifferSnapshotInfo> snapshots = new ArrayList<>();
+
+  /**
+   * Graph type.
+   */
+  enum GType {
+    FNAME,
+    KEYSIZE,
+    CUMUTATIVE_SIZE
+  }
+
+  @BeforeAll
+  public static void init() {
+    // Set differ log level
+    GenericTestUtils.setLogLevel(RocksDBCheckpointDiffer.getLog(), Level.INFO);
+    // Set test class log level
+    GenericTestUtils.setLogLevel(TestRocksDBCheckpointDiffer.LOG, Level.INFO);
+  }
 
-  // keeps track of all the snapshots created so far.
-  private static int lastSnapshotCounter = 0;
-  private static String lastSnapshotPrefix = "snap_id_";
+  @Test

Review Comment:
   Thanks. Added assertions.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] prashantpogde commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012354315


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -215,253 +304,252 @@ public static void addDebugLevel(Integer level) {
     DEBUG_LEVEL.add(level);
   }
 
-  // Flushes the WAL and Creates a RocksDB checkpoint
-  @SuppressWarnings({"NM_METHOD_NAMING_CONVENTION"})
-  public void createCheckPoint(String dbPathArg, String cpPathArg,
-                               RocksDB rocksDB) {
-    LOG.warn("Creating Checkpoint for RocksDB instance : " +
-        dbPathArg + "in a CheckPoint Location" + cpPathArg);
-    try {
-      rocksDB.flush(new FlushOptions());
-      Checkpoint cp = Checkpoint.create(rocksDB);
-      cp.createCheckpoint(cpPathArg);
-    } catch (RocksDBException e) {
-      throw new RuntimeException(e.getMessage());
+  /**
+   * Append (then flush) to the current compaction log file path.
+   * Note: This does NOT automatically append newline to the log.
+   */
+  private synchronized void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogPath == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Compaction log path is not set. "
+          + "Please check initialization.");
+      throw new RuntimeException("Compaction log path not set");
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogPath),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogPath, e);
     }
   }
 
-  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
-      throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
-            }
-          }
-        };
+  /**
+   * Append a sequence number to the compaction log (roughly) when an Ozone
+   * snapshot (RDB checkpoint) is taken.
+   * @param sequenceNum RDB sequence number
+   */
+  public void appendSequenceNumberToCompactionLog(long sequenceNum) {
+    final String line = COMPACTION_LOG_SEQNUM_LINE_PREFIX + sequenceNum + "\n";
+    appendToCurrentCompactionLog(line);
+  }
 
-    list.add(onCompactionCompletedListener);
+  /**
+   * Takes {@link org.rocksdb.Options}.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
     rocksOptions.setListeners(list);
   }
 
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
 
+  /**
+   * Takes {@link org.rocksdb.DBOptions}.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
 
-  public void setRocksDBForCompactionTracking(Options rocksOptions)
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      Options rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db,final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file);
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix,
-                      numKeys, currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix, numKeys,
-                        UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null,
-                    compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().size() == 0) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          // Create hardlink backups for the SST files that are going
+          // to be deleted after this RDB compaction.
+          for (String file : compactionJobInfo.inputFiles()) {
+            LOG.debug("Creating hard link for '{}'", file);
+            String saveLinkFileName =
+                sstBackupDir + new File(file).getName();
+            Path link = Paths.get(saveLinkFileName);
+            Path srcFile = Paths.get(file);
+            try {
+              Files.createLink(link, srcFile);
+            } catch (IOException e) {
+              LOG.error("Exception in creating hard link for {}", file);
+              throw new RuntimeException("Failed to create hard link", e);
             }
           }
-        };
 
-    list.add(onCompactionCompletedListener);
-    rocksOptions.setListeners(list);
+        }
+      }
+    };
   }
 
-  public RocksDB getRocksDBInstanceWithCompactionTracking(String dbPath)
-      throws RocksDBException {
-    final Options opt = new Options().setCreateIfMissing(true)
-        .setCompressionType(CompressionType.NO_COMPRESSION);
-    opt.setMaxBytesForLevelMultiplier(2);
-    setRocksDBForCompactionTracking(opt);
-    return RocksDB.open(opt, dbPath);
+  private AbstractEventListener newCompactionCompletedListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionCompleted(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().isEmpty()) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          final StringBuilder sb = new StringBuilder();
+
+          if (LOG.isDebugEnabled()) {
+            // Print compaction reason for this entry in the log file
+            // e.g. kLevelL0FilesNum / kLevelMaxLevelSize.
+            sb.append(COMPACTION_LOG_COMMENT_LINE_PREFIX)
+                .append(compactionJobInfo.compactionReason())
+                .append('\n');
+          }
+
+          // Mark the beginning of a compaction log
+          sb.append(COMPACTION_LOG_ENTRY_LINE_PREFIX);
+
+          // Trim DB path, only keep the SST file name
+          final int filenameOffset =
+              compactionJobInfo.inputFiles().get(0).lastIndexOf("/") + 1;
+
+          // Append the list of input files
+          final List<String> inputFiles = compactionJobInfo.inputFiles();
+          // Trim the file path, leave only the SST file name without extension
+          inputFiles.replaceAll(s -> s.substring(
+              filenameOffset, s.length() - SST_FILE_EXTENSION_LENGTH));
+          final String inputFilesJoined = String.join(",", inputFiles);
+          sb.append(inputFilesJoined);
+
+          // Insert delimiter between input files an output files
+          sb.append(':');
+
+          // Append the list of output files
+          final List<String> outputFiles = compactionJobInfo.outputFiles();
+          outputFiles.replaceAll(s -> s.substring(
+              filenameOffset, s.length() - SST_FILE_EXTENSION_LENGTH));
+          final String outputFilesJoined = String.join(",", outputFiles);
+          sb.append(outputFilesJoined);
+
+          // End of line
+          sb.append('\n');
+
+          // Write input and output file names to compaction log
+          appendToCurrentCompactionLog(sb.toString());
+
+          // Populate the DAG
+          populateCompactionDAG(inputFiles, outputFiles,
+              db.getLatestSequenceNumber());
+/*
+          if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
+            printMutableGraph(null, null, compactionDAGFwd);
+          }
+ */
+        }
+      }
+    };
   }
 
-  // Get a summary of a given SST file
-  public long getSSTFileSummary(String filename)
-      throws RocksDBException {
+  /**
+   * Get number of keys in an SST file.
+   * @param filename SST filename
+   * @return number of keys
+   */
+  private long getSSTFileSummary(String filename) throws RocksDBException {
+
+    if (!filename.endsWith(SST_FILE_EXTENSION)) {
+      filename += SST_FILE_EXTENSION;
+    }
+
     Options option = new Options();
     SstFileReader reader = new SstFileReader(option);
-    try {
-      reader.open(saveCompactedFilePath + filename);
-    } catch (RocksDBException e) {
-      reader.open(rocksDbPath + "/"+ filename);
+
+    File sstFile = new File(sstBackupDir + filename);
+    File sstFileInActiveDB = new File(activeDBLocationStr + filename);
+    if (sstFile.exists()) {
+      reader.open(sstBackupDir + filename);
+    } else if (sstFileInActiveDB.exists()) {
+      reader.open(activeDBLocationStr + filename);
+    } else {
+      throw new RuntimeException("Can't find SST file: " + filename);
     }
+
     TableProperties properties = reader.getTableProperties();
-    LOG.warn("getSSTFileSummary " + filename + ":: " +
-        properties.getNumEntries());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("{} has {} keys", filename, properties.getNumEntries());
+    }
     return properties.getNumEntries();
   }
 
-  // Read the current Live manifest for a given RocksDB instance (Active or
-  // Checkpoint). Returns the list of currently active SST FileNames.
-  @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
+  /**
+   * Helper method to trim the filename retrieved from LiveFileMetaData.
+   */
+  private String trimSSTFilename(String filename) {
+    if (!filename.startsWith("/")) {
+      final String errorMsg = String.format(
+          "Invalid start of filename: '%s'. Expected '/'", filename);
+      LOG.error(errorMsg);
+      throw new RuntimeException(errorMsg);
+    }
+    if (!filename.endsWith(SST_FILE_EXTENSION)) {
+      final String errorMsg = String.format(
+          "Invalid extension of file: '%s'. Expected '%s'",
+          filename, SST_FILE_EXTENSION_LENGTH);
+      LOG.error(errorMsg);
+      throw new RuntimeException(errorMsg);
+    }
+    return filename.substring("/".length(),
+        filename.length() - SST_FILE_EXTENSION_LENGTH);
+  }
+
+  /**
+   * Read the current Live manifest for a given RocksDB instance (Active or
+   * Checkpoint).
+   * @param dbPathArg path to a RocksDB directory
+   * @return a list of SST files (without extension) in the DB.
+   */
   public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     RocksDB rocksDB = null;
     HashSet<String> liveFiles = new HashSet<>();
-    //
-    try (final Options options =
-             new Options().setParanoidChecks(true)
-                 .setCreateIfMissing(true)
-                 .setCompressionType(CompressionType.NO_COMPRESSION)
-                 .setForceConsistencyChecks(false)) {
-      rocksDB = RocksDB.openReadOnly(options, dbPathArg);
+
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+    final List<ColumnFamilyDescriptor> cfd = new ArrayList<>();
+    cfd.add(new ColumnFamilyDescriptor(
+        "keyTable".getBytes(StandardCharsets.UTF_8)));

Review Comment:
   I believe we have separate task items for tracking support for FSO buckets. Let us Add DirectoryTable and FileTable related changes in a separate PR. 



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993140147


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,13 +529,151 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   * TODO: Drop synchronized? and make this thread safe?
+   */
+  private synchronized void processCompactionLogLine(String line) {
+    LOG.info("Processing line: {}", line);
+
+    // Skip comments
+    if (line.startsWith("#")) {
+      LOG.info("Skipped comment.");
+      return;
+    }
+
+    if (sstTokensRead == null) {
+      // Store the tokens in the first line
+      sstTokensRead = line.split("\t");
+      LOG.info("Length of inputFiles = {}", sstTokensRead.length);
+      if (sstTokensRead.length == 0) {
+        // Sanity check. inputFiles should never be empty. outputFiles could.
+        throw new RuntimeException(
+            "Compaction inputFiles list is empty. File is corrupted?");
+      }
+    } else {
+      final String[] outputFilesRead = line.split("\t");
+      LOG.info("Length of outputFiles = {}", outputFilesRead.length);
+
+      // Populate the compaction DAG
+      populateCompactionDAG(sstTokensRead, outputFilesRead);
+
+      // Reset inputFilesRead to null so
+      sstTokensRead = null;
+    }
+  }
+
+  private String[] sstTokensRead;
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   * <p>
+   * DO NOT use this function in another context without understanding what it
+   * does, due to the state preserved between calls (in sstTokensRead).
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> stream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      assert (sstTokensRead == null);
+      stream.forEach(this::processCompactionLogLine);
+      if (sstTokensRead != null) {
+        // Sanity check. Temp variable must be null after parsing.
+        // Otherwise it means the compaction log is corrupted.
+        throw new RuntimeException("Missing output files line. Corrupted?");
+      }
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Read compaction log until all dest (and src) db checkpoint SST
+   * nodes show up in the graph, or when it reaches the end of the log.
+   */
+  private boolean loadCompactionDAGBySSTSet(HashSet<String> sstSet) {
+
+    // Make a copy of the sstSet
+    HashSet<String> loadSet = new HashSet<>(sstSet);
+
+    // Check if all the nodes in the provided SST set is already loaded in DAG
+    for (String sstFile : sstSet) {
+      if (compactionNodeTable.containsKey(sstFile)) {
+        loadSet.remove(sstFile);
+      }
+    }
+
+    if (loadSet.size() == 0) {
+      // All expected nodes in the sstSet are already there,
+      //  no need to read any compaction log
+      return true;
+    }
+
+    // Otherwise, load compaction logs in order until all nodes are present in
+    //  the DAG.
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase().endsWith(".log"))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+
+          // TODO: Potential optimization: stop reading as soon as all nodes are
+          //  there. Currently it loads an entire file at a time.
+          readCompactionLogToDAG(logPath.toString());
+
+          for (Iterator<String> it = loadSet.iterator(); it.hasNext();) {
+            String sstFile = it.next();
+            if (compactionNodeTable.containsKey(sstFile)) {
+              LOG.debug("Found SST node: {}", sstFile);
+              it.remove();
+            }
+          }
+
+          if (loadSet.size() == 0) {
+            break;
+          }
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+
+    // Just in case there are still nodes to be expected not loaded.
+    if (loadSet.size() > 0) {
+      LOG.warn("The following nodes are missing from the compaction log: {}. "
+          + "Possibly because those a newly flushed SSTs that haven't gone "
+          + "though any compaction yet", loadSet);
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Given the src and destination Snapshots, it prints a Diff list.
+   *
+   * Expected input: src is a checkpoint taken AFTER dest checkpoint.
+   *
+   * @param src
+   * @param dest
+   * @throws RocksDBException
+   */
+  private synchronized List<String> printSnapdiffSSTFiles(
+      Snapshot src, Snapshot dest) {
+
+    LOG.warn("src db checkpoint: {}", src.dbPath);  // from
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
+    LOG.warn("dest db checkpoint: {}", dest.dbPath);  //to
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
+    System.out.println();
+
+    // Read compaction log until all dest (and src) db checkpoint SST
+    // nodes show up in the graph
+//    loadCompactionDAGBySSTSet(destSnapFiles);

Review Comment:
   Yes this is intentional because in reality it is redundant under the current assumption as noted in the javadoc:
   
   > Expected input: src is a checkpoint taken AFTER dest checkpoint
   
   When loading `srcSnapFiles`, `destSnapFiles` should have been traversed already.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on PR #3824:
URL: https://github.com/apache/ozone/pull/3824#issuecomment-1276821184

   > This is in response to the discussion here: [#3786 (comment)](https://github.com/apache/ozone/pull/3786#discussion_r992935645)
   > 
   > To me there is value in the conceptual simplicity of having all the state in rocksdb.
   > 
   > And I don't think it would have to be a new RDB instance. It could just be a separate CF in the existing one. I know that that CF wouldn't be shared through ratis transactions, but it would be shared with followers when they get bootstrapped (which happens to be what we want.)
   
   Doesn't bootstrapping pack the entire `ozone-metadata` directory? If this is the case then the new `compaction-log` directory would have been included automatically. (I need to double check.)
   
   I understand it would be easier (and cleaner) if we just use a new CF in the **existing** OM RDB to store compaction history. The implementation would be trivial. But there is some aspects I dislike about it:
   
   1. The info we are storing to the DB (input / output SST file pairs) tightly relates to the file structure of the database itself, which isn't really related to main functions of OM, and can differ from OM to OM. Plus the fact that the compaction log isn't crucial for the RocksDiff feature to work (as it is meant for acceleration by returning only the relevant set of SST files). If we do want to persist the log in RDB I would prefer starting a new RDB instance just for this (a metadata DB of the existing DB).
   2. We are putting extra pressure on RDB compaction by writing those extra K-V pairs to the DB **itself** after each compaction. Though the pressure should be small provided that compaction doesn't happen too frequently (depends on DB write).
   
   > Finally, with respect to the ordering problem mentioned above, couldn't the db lastSeqNum be used as the rocksdb key, (as it is used in the filename currently)?
   
   Good point.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993956506


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -129,6 +201,24 @@ public RocksDBCheckpointDiffer(String dbPath,
     cpPath = checkpointPath;
 
     saveCompactedFilePath = sstFileSaveDir;
+
+    // Append /
+    if (!saveCompactedFilePath.endsWith("/")) {
+      saveCompactedFilePath += "/";
+    }
+
+    // Create the directory if SST backup path does not already exist
+    File dir = new File(saveCompactedFilePath);
+    if (dir.exists()) {
+      deleteDirectory(dir);  // TODO: FOR EASE OF TESTING ONLY. DO NOT DELETE DIR WHEN MERGING
+    }
+    if (!dir.mkdir()) {
+      LOG.error("Failed to create SST file backup directory!");
+      // TODO: Throw unrecoverable exception and Crash OM ?
+      throw new RuntimeException("Failed to create SST file backup directory. "
+          + "Check write permission.");
+    }
+
     rocksDbPath = dbPath;
     cfDBPath = cfPath;

Review Comment:
   this seems unused.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012542936


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -215,390 +288,511 @@ public static void addDebugLevel(Integer level) {
     DEBUG_LEVEL.add(level);
   }
 
-  // Flushes the WAL and Creates a RocksDB checkpoint
-  @SuppressWarnings({"NM_METHOD_NAMING_CONVENTION"})
-  public void createCheckPoint(String dbPathArg, String cpPathArg,
-                               RocksDB rocksDB) {
-    LOG.warn("Creating Checkpoint for RocksDB instance : " +
-        dbPathArg + "in a CheckPoint Location" + cpPathArg);
-    try {
-      rocksDB.flush(new FlushOptions());
-      Checkpoint cp = Checkpoint.create(rocksDB);
-      cp.createCheckpoint(cpPathArg);
-    } catch (RocksDBException e) {
-      throw new RuntimeException(e.getMessage());
+  /**
+   * Append (then flush) to the current compaction log file path.
+   * Note: This does NOT automatically append newline to the log.
+   */
+  private synchronized void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogPath == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Compaction log path is not set. "
+          + "Please check initialization.");
+      throw new RuntimeException("Compaction log path not set");
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogPath),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogPath, e);
     }
   }
 
-  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
-      throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
-            }
-          }
-        };
+  /**
+   * Append a sequence number to the compaction log (roughly) when an Ozone
+   * snapshot (RDB checkpoint) is taken.
+   * @param sequenceNum RDB sequence number
+   */
+  public void appendSequenceNumberToCompactionLog(long sequenceNum) {
+    final String line = COMPACTION_LOG_SEQNUM_LINE_PREFIX + sequenceNum + "\n";
+    appendToCurrentCompactionLog(line);
+  }
 
-    list.add(onCompactionCompletedListener);
+  /**
+   * Takes {@link org.rocksdb.Options}.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
     rocksOptions.setListeners(list);
   }
 
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
 
+  /**
+   * Takes {@link org.rocksdb.DBOptions}.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
 
-  public void setRocksDBForCompactionTracking(Options rocksOptions)
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      Options rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db,final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file);
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix,
-                      numKeys, currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix, numKeys,
-                        UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null,
-                    compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().size() == 0) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          // Create hardlink backups for the SST files that are going
+          // to be deleted after this RDB compaction.
+          for (String file : compactionJobInfo.inputFiles()) {
+            LOG.debug("Creating hard link for '{}'", file);
+            String saveLinkFileName =
+                sstBackupDir + new File(file).getName();
+            Path link = Paths.get(saveLinkFileName);
+            Path srcFile = Paths.get(file);
+            try {
+              Files.createLink(link, srcFile);
+            } catch (IOException e) {
+              LOG.error("Exception in creating hard link for {}", file);
+              throw new RuntimeException("Failed to create hard link", e);
             }
           }
-        };
 
-    list.add(onCompactionCompletedListener);
-    rocksOptions.setListeners(list);
+        }
+      }
+    };
   }
 
-  public RocksDB getRocksDBInstanceWithCompactionTracking(String dbPath)
-      throws RocksDBException {
-    final Options opt = new Options().setCreateIfMissing(true)
-        .setCompressionType(CompressionType.NO_COMPRESSION);
-    opt.setMaxBytesForLevelMultiplier(2);
-    setRocksDBForCompactionTracking(opt);
-    return RocksDB.open(opt, dbPath);
+  private AbstractEventListener newCompactionCompletedListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionCompleted(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().isEmpty()) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          final StringBuilder sb = new StringBuilder();
+
+          if (LOG.isDebugEnabled()) {
+            // Print compaction reason for this entry in the log file
+            // e.g. kLevelL0FilesNum / kLevelMaxLevelSize.
+            sb.append(COMPACTION_LOG_COMMENT_LINE_PREFIX)
+                .append(compactionJobInfo.compactionReason())
+                .append('\n');
+          }
+
+          // Mark the beginning of a compaction log
+          sb.append(COMPACTION_LOG_ENTRY_LINE_PREFIX);
+
+          // Trim DB path, only keep the SST file name

Review Comment:
   The full path is highly redundant. IMO there is no need to persist the full path as the SST file can only either be in the SST backup dir or in active DB dir. Storing the full path easily doubles the in-memory map memory consumption compared to file name only.
   
   Also when this listener is triggered, the input / output SST file path is always under the active DB. But those SST files may well have been deleted from active DB instance and the SSTs are only kept under the backup dir.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on PR #3824:
URL: https://github.com/apache/ozone/pull/3824#issuecomment-1302383295

   Thanks @GeorgeJahad @sadanand48 @prashantpogde @hemantk-12 for reviewing this.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1011109396


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,134 +558,231 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   */
+  private synchronized void processCompactionLogLine(String line) {
+
+    LOG.debug("Processing line: {}", line);
+
+    if (line.startsWith("#")) {
+      // Skip comments
+      LOG.debug("Comment line, skipped");
+    } else if (line.startsWith(COMPACTION_LOG_SEQNUM_LINE_PREFIX)) {
+      // Read sequence number
+      LOG.debug("Reading sequence number as snapshot generation");
+      final String seqNumStr =
+          line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length()).trim();
+      // This would the snapshot generation for the nodes to come
+      reconstructionSnapshotGeneration = Long.parseLong(seqNumStr);
+    } else if (line.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) {
+      // Read compaction log entry
+
+      // Trim the beginning
+      line = line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length());
+      final String[] io = line.split(":");
+      if (io.length != 2) {
+        LOG.error("Invalid line in compaction log: {}", line);
+        return;
+      }
+      final String[] inputFiles = io[0].split(",");
+      final String[] outputFiles = io[1].split(",");
+      populateCompactionDAG(Arrays.asList(inputFiles),
+          Arrays.asList(outputFiles), reconstructionSnapshotGeneration);
+    } else {
+      LOG.error("Invalid line in compaction log: {}", line);
+    }
+  }
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> logLineStream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      logLineStream.forEach(this::processCompactionLogLine);
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Load existing compaction log files to the in-memory DAG.
+   * This only needs to be done once during OM startup.
+   */
+  public synchronized void loadAllCompactionLogs() {
+    if (compactionLogDir == null) {
+      throw new RuntimeException("Compaction log directory must be set first");
+    }
+    reconstructionSnapshotGeneration = 0L;
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase()
+              .endsWith(COMPACTION_LOG_FILENAME_SUFFIX))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+          readCompactionLogToDAG(logPath.toString());
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+  }
+
+  /**
+   * Snapshot information node class in the DAG.
+   */
+  static class Snapshot {
+    private final String dbPath;
+    private final String snapshotID;
+    private final long snapshotGeneration;
+
+    Snapshot(String db, String id, long gen) {
+      dbPath = db;
+      snapshotID = id;
+      snapshotGeneration = gen;
+    }
+
+    public String getDbPath() {
+      return dbPath;
+    }
+
+    public String getSnapshotID() {
+      return snapshotID;
+    }
+
+    public long getSnapshotGeneration() {
+      return snapshotGeneration;
+    }
+  }
+
+  /**
+   * Get a list of SST files that differs between src and destination snapshots.
+   * <p>
+   * Expected input: src is a snapshot taken AFTER the dest.
+   *
+   * @param src source snapshot
+   * @param dest destination snapshot
+   */
+  public synchronized List<String> getSSTDiffList(Snapshot src, Snapshot dest) {
+
+    LOG.debug("src '{}' -> dest '{}'", src.dbPath, dest.dbPath);
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
 
     HashSet<String> fwdDAGSameFiles = new HashSet<>();
     HashSet<String> fwdDAGDifferentFiles = new HashSet<>();
 
-    LOG.warn("Doing forward diff between source and destination " +
-        "Snapshots:" + src.dbPath + ", " + dest.dbPath);
-    realPrintSnapdiffSSTFiles(src, dest, srcSnapFiles, destSnapFiles,
-        compactionDAGFwd,
-        fwdDAGSameFiles,
-        fwdDAGDifferentFiles);
+    LOG.debug("Doing forward diff between src and dest snapshots: " +
+        src.dbPath + " to " + dest.dbPath);
+    internalGetSSTDiffList(src, dest, srcSnapFiles, destSnapFiles,
+        compactionDAGFwd, fwdDAGSameFiles, fwdDAGDifferentFiles);
 
-    LOG.warn("Overall Summary \n" +
-            "Doing Overall diff between source and destination Snapshots:" +
-        src.dbPath + ", " + dest.dbPath);
-    System.out.print("fwd DAG Same files :");
-    for (String file : fwdDAGSameFiles) {
-      System.out.print(file + ", ");
-    }
-    LOG.warn("");
-    System.out.print("\nFwd DAG Different files :");
-    for (String file : fwdDAGDifferentFiles) {
-      CompactionNode n = compactionNodeTable.get(file);
-      System.out.print(file + ", ");
+    List<String> res = new ArrayList<>();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Result of diff from src '" + src.dbPath + "' to dest '" +
+          dest.dbPath + "':");
+      StringBuilder logSB = new StringBuilder();
+
+      logSB.append("Fwd DAG same SST files:      ");
+      for (String file : fwdDAGSameFiles) {
+        logSB.append(file).append(" ");
+      }
+      LOG.debug(logSB.toString());
+
+      logSB.setLength(0);
+      logSB.append("Fwd DAG different SST files: ");
+      for (String file : fwdDAGDifferentFiles) {
+        logSB.append(file).append(" ");
+        res.add(file);
+      }
+      LOG.debug(logSB.toString());
+
+    } else {
+      res.addAll(fwdDAGDifferentFiles);
     }
-    LOG.warn("");
+
+    return res;
   }
 
+  /**
+   * Core getSSTDiffList logic.
+   */
   @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-  public synchronized void realPrintSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest,
-      HashSet<String> srcSnapFiles,
-      HashSet<String> destSnapFiles,
+  private void internalGetSSTDiffList(Snapshot src, Snapshot dest,
+      HashSet<String> srcSnapFiles, HashSet<String> destSnapFiles,
       MutableGraph<CompactionNode> mutableGraph,
       HashSet<String> sameFiles, HashSet<String> differentFiles) {
 
-
     for (String fileName : srcSnapFiles) {
       if (destSnapFiles.contains(fileName)) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + " and Dest " +
-            "Snapshot" + dest.dbPath + " Contain Same file " + fileName);
+        LOG.debug("Source '{}' and destination '{}' share the same SST '{}'",
+            src.dbPath, dest.dbPath, fileName);
         sameFiles.add(fileName);
         continue;
       }
       CompactionNode infileNode =
           compactionNodeTable.get(Paths.get(fileName).getFileName().toString());
       if (infileNode == null) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + "File " + fileName + "was " +
-            "never compacted");
+        LOG.debug("Src " + src.dbPath + " File " + fileName +
+            " was never compacted");
         differentFiles.add(fileName);
         continue;
       }
-      System.out.print(" Expandin File:" + fileName + ":\n");
-      Set<CompactionNode> nextLevel = new HashSet<>();
-      nextLevel.add(infileNode);
+      LOG.debug("Expanding SST file: " + fileName);
       Set<CompactionNode> currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
-      nextLevel = new HashSet<>();
+      currentLevel.add(infileNode);
+      Set<CompactionNode> nextLevel = new HashSet<>();
       int i = 1;
       while (currentLevel.size() != 0) {
-        LOG.warn("DAG Level :" + i++);
+        LOG.debug("DAG Level: " + i++);
         for (CompactionNode current : currentLevel) {
-          LOG.warn("acknowledging file " + current.fileName);
+          LOG.debug("Acknowledging file " + current.fileName);
           if (current.snapshotGeneration <= dest.snapshotGeneration) {
-            LOG.warn("Reached dest generation count. SrcSnapshot : " +
-                src.dbPath + " and Dest " + "Snapshot" + dest.dbPath +
-                " Contain Diffrent file " + current.fileName);
+            LOG.debug("Reached dest generation count. Src: " +
+                src.dbPath + " and Dest: " + dest.dbPath +
+                " have different file: " + current.fileName);
             differentFiles.add(current.fileName);
             continue;
           }
           Set<CompactionNode> successors = mutableGraph.successors(current);
-          if (successors == null || successors.size() == 0) {
-            LOG.warn("No further compaction for the file" +
-                ".SrcSnapshot : " + src.dbPath + " and Dest " +
-                "Snapshot" + dest.dbPath + " Contain Diffrent file " +
-                current.fileName);
+          if (successors.size() == 0) {

Review Comment:
   > Is `successors` nullable?
   
   Good point. But doesn't seem so according to Guava `Graph` [javadoc](https://www.javadoc.io/doc/com.google.guava/guava/31.1-jre/com/google/common/graph/Graph.html#successors(N)).
   
   Suggestions other than this would been included in the next commit. Thx!



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1011113296


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -608,52 +792,48 @@ public Comparator<CompactionNode> reversed() {
     }
   }
 
-
-  public void dumpCompactioNodeTable() {
-    List<CompactionNode> nodeList =
-        compactionNodeTable.values().stream().collect(Collectors.toList());
-    Collections.sort(nodeList, new NodeComparator());
-    for (CompactionNode n : nodeList ) {
-      LOG.warn("File : " + n.fileName + " :: Total keys : "
-          + n.totalNumberOfKeys);
-      LOG.warn("File : " + n.fileName + " :: Cumulative keys : "  +
+  @VisibleForTesting
+  public void dumpCompactionNodeTable() {
+    List<CompactionNode> nodeList = compactionNodeTable.values().stream()
+        .sorted(new NodeComparator()).collect(Collectors.toList());
+    for (CompactionNode n : nodeList) {
+      LOG.info("File '{}' total keys: {}", n.fileName, n.totalNumberOfKeys);
+      LOG.info("File '{}' cumulative keys: {}", n.fileName,
           n.cumulativeKeysReverseTraversal);
     }
   }
 
+  @VisibleForTesting
   @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-  public synchronized void printMutableGraphFromAGivenNode(
-      String fileName, int level, MutableGraph<CompactionNode> mutableGraph) {
+  public synchronized void printMutableGraphFromAGivenNode(String fileName,
+      int level, MutableGraph<CompactionNode> mutableGraph) {
     CompactionNode infileNode =
         compactionNodeTable.get(Paths.get(fileName).getFileName().toString());
     if (infileNode == null) {
       return;
     }
-    System.out.print("\nCompaction Level : " + level + " Expandin File:" +
-        fileName + ":\n");
+    LOG.info("\nCompaction Level: " + level + " Expanding File: " + fileName);
     Set<CompactionNode> nextLevel = new HashSet<>();
     nextLevel.add(infileNode);
-    Set<CompactionNode> currentLevel = new HashSet<>();
-    currentLevel.addAll(nextLevel);
+    Set<CompactionNode> currentLevel = new HashSet<>(nextLevel);
     int i = 1;
     while (currentLevel.size() != 0) {
-      LOG.warn("DAG Level :" + i++);
+      LOG.info("DAG Level: " + i++);
+      StringBuilder sb = new StringBuilder();
       for (CompactionNode current : currentLevel) {
         Set<CompactionNode> successors = mutableGraph.successors(current);
         for (CompactionNode oneSucc : successors) {
-          System.out.print(oneSucc.fileName + " ");
+          sb.append(oneSucc.fileName).append(" ");
           nextLevel.add(oneSucc);
         }
       }
-      currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
+      LOG.info(sb.toString());
+      currentLevel = new HashSet<>(nextLevel);
       nextLevel = new HashSet<>();
-      LOG.warn("");
     }
   }
 
-  public synchronized void printMutableGraph(
-      String srcSnapId, String destSnapId,
+  synchronized void printMutableGraph(String srcSnapId, String destSnapId,
       MutableGraph<CompactionNode> mutableGraph) {
     LOG.warn("Printing the Graph");

Review Comment:
   Good catch. Fixed



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] hemantk-12 commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
hemantk-12 commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012198808


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -215,253 +305,265 @@ public static void addDebugLevel(Integer level) {
     DEBUG_LEVEL.add(level);
   }
 
-  // Flushes the WAL and Creates a RocksDB checkpoint
-  @SuppressWarnings({"NM_METHOD_NAMING_CONVENTION"})
-  public void createCheckPoint(String dbPathArg, String cpPathArg,
-                               RocksDB rocksDB) {
-    LOG.warn("Creating Checkpoint for RocksDB instance : " +
-        dbPathArg + "in a CheckPoint Location" + cpPathArg);
-    try {
-      rocksDB.flush(new FlushOptions());
-      Checkpoint cp = Checkpoint.create(rocksDB);
-      cp.createCheckpoint(cpPathArg);
-    } catch (RocksDBException e) {
-      throw new RuntimeException(e.getMessage());
+  /**
+   * Append (then flush) to the current compaction log file path.
+   * Note: This does NOT automatically append newline to the log.
+   */
+  private synchronized void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogPath == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Compaction log path is not set. "
+          + "Please check initialization.");
+      throw new RuntimeException("Compaction log path not set");
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogPath),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogPath, e);
     }
   }
 
-  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
-      throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
-            }
-          }
-        };
+  /**
+   * Append a sequence number to the compaction log (roughly) when an Ozone
+   * snapshot (RDB checkpoint) is taken.
+   * @param sequenceNum RDB sequence number
+   */
+  public void appendSequenceNumberToCompactionLog(long sequenceNum) {
+    final String line = COMPACTION_LOG_SEQNUM_LINE_PREFIX + sequenceNum + "\n";
+    appendToCurrentCompactionLog(line);
+  }
 
-    list.add(onCompactionCompletedListener);
+  /**
+   * Takes {@link org.rocksdb.Options}.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
     rocksOptions.setListeners(list);
   }
 
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
 
+  /**
+   * Takes {@link org.rocksdb.DBOptions}.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
 
-  public void setRocksDBForCompactionTracking(Options rocksOptions)
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      Options rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db,final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file);
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix,
-                      numKeys, currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix, numKeys,
-                        UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null,
-                    compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().size() == 0) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          // Create hardlink backups for the SST files that are going
+          // to be deleted after this RDB compaction.
+          for (String file : compactionJobInfo.inputFiles()) {
+            LOG.debug("Creating hard link for '{}'", file);
+            String saveLinkFileName =
+                sstBackupDir + new File(file).getName();
+            Path link = Paths.get(saveLinkFileName);
+            Path srcFile = Paths.get(file);
+            try {
+              Files.createLink(link, srcFile);
+            } catch (IOException e) {
+              LOG.error("Exception in creating hard link for {}", file);
+              throw new RuntimeException("Failed to create hard link", e);
             }
           }
-        };
 
-    list.add(onCompactionCompletedListener);
-    rocksOptions.setListeners(list);
+        }
+      }
+    };
   }
 
-  public RocksDB getRocksDBInstanceWithCompactionTracking(String dbPath)
-      throws RocksDBException {
-    final Options opt = new Options().setCreateIfMissing(true)
-        .setCompressionType(CompressionType.NO_COMPRESSION);
-    opt.setMaxBytesForLevelMultiplier(2);
-    setRocksDBForCompactionTracking(opt);
-    return RocksDB.open(opt, dbPath);
+  /**
+   * Helper function to append the list of SST files to a StringBuilder
+   * for a compaction log entry. Does not append a new line.
+   */
+  private static void appendCompactionLogStringBuilder(List<String> files,
+      StringBuilder sb) {
+
+    Iterator<String> it = files.iterator();
+    while (it.hasNext()) {

Review Comment:
   nit: You can use `String.join(",", files);` or `files.stream().collect(Collectors.joining(","));`.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1009732852


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -230,196 +330,154 @@ public void createCheckPoint(String dbPathArg, String cpPathArg,
     }
   }
 
+  /**
+   * Append (then flush) to the current compaction log file path.
+   */
+  public void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogFilename == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Current compaction log filename is not set. "
+          + "Please check initialization.");
+      return;
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogFilename),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogFilename, e);
+    }
+  }
+
+  /**
+   * This takes Options.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  /**
+   * This takes DBOptions.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
   public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (db) {

Review Comment:
   I agree we need to synchronize.  My question is why are we using the "db" object?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1009776885


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -230,196 +330,154 @@ public void createCheckPoint(String dbPathArg, String cpPathArg,
     }
   }
 
+  /**
+   * Append (then flush) to the current compaction log file path.
+   */
+  public void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogFilename == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Current compaction log filename is not set. "
+          + "Please check initialization.");
+      return;
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogFilename),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogFilename, e);
+    }
+  }
+
+  /**
+   * This takes Options.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  /**
+   * This takes DBOptions.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
   public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (db) {

Review Comment:
   Yup let me switch to synchronizing on a dummy Java object instead.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] hemantk-12 commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
hemantk-12 commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1011029835


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,134 +611,287 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  private long reconstructionSnapshotGeneration;
+
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   */
+  private synchronized void processCompactionLogLine(String line) {
+
+    LOG.debug("Processing line: {}", line);
+
+    if (line.startsWith("#")) {
+      // Skip comments
+      LOG.debug("Comment line, skipped");
+    } else if (line.startsWith(COMPACTION_LOG_SEQNUM_LINE_PREFIX)) {
+      // Read sequence number
+      LOG.debug("Reading sequence number as snapshot generation");
+      final String seqNumStr =
+          line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length()).trim();
+      // This would the snapshot generation for the nodes to come
+      reconstructionSnapshotGeneration = Long.parseLong(seqNumStr);
+    } else if (line.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) {
+      // Read compaction log entry
+
+      // Trim the beginning
+      line = line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length());
+      final String[] io = line.split(":");
+      if (io.length != 2) {
+        LOG.error("Invalid line in compaction log: {}", line);
+        return;
+      }
+      final String[] inputFiles = io[0].split(",");
+      final String[] outputFiles = io[1].split(",");
+      populateCompactionDAG(Arrays.asList(inputFiles),
+          Arrays.asList(outputFiles), reconstructionSnapshotGeneration);
+    } else {
+      LOG.error("Invalid line in compaction log: {}", line);
+    }
+  }
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> logLineStream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      logLineStream.forEach(this::processCompactionLogLine);
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Returns a set of SST nodes that doesn't exist in the in-memory DAG.
+   */
+  private Set<String> getNonExistentSSTSet(Set<String> sstSet) {
+
+    // Make a copy of sstSet
+    HashSet<String> loadSet = new HashSet<>(sstSet);
+
+    // Check if all the nodes in the provided SST set is already loaded in DAG
+    for (String sstFile : sstSet) {
+      if (compactionNodeTable.containsKey(sstFile)) {
+        loadSet.remove(sstFile);
+      }
+    }
+
+    return loadSet;
+  }
+
+  /**
+   * Returns true only when all nodes in sstSet exists in DAG.
+   */
+  private boolean isSSTSetLoaded(HashSet<String> sstSet) {
+
+    return getNonExistentSSTSet(sstSet).size() == 0;
+  }
+
+  /**
+   * Read compaction log until all dest (and src) db checkpoint SST
+   * nodes show up in the graph, or when it reaches the end of the log.
+   */
+  private boolean loadCompactionDAGBySSTSet(HashSet<String> sstSet) {
+
+    // Get a set of SSTs that doesn't exist in the current in-memory DAG
+    Set<String> loadSet = getNonExistentSSTSet(sstSet);
+
+    if (loadSet.size() == 0) {
+      // All expected nodes in the sstSet are already there,
+      //  no need to read/load any compaction log from disk.
+      return true;
+    }
+
+    // Otherwise, load compaction logs in order until all nodes are present in
+    //  the DAG.
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString()
+              .toLowerCase().endsWith(COMPACTION_LOG_FILENAME_SUFFIX))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+
+          // TODO: Potential optimization: stop reading as soon as all nodes are
+          //  there. Currently it loads an entire file at a time.
+          readCompactionLogToDAG(logPath.toString());
+
+          for (Iterator<String> it = loadSet.iterator(); it.hasNext();) {
+            String sstFile = it.next();
+            if (compactionNodeTable.containsKey(sstFile)) {
+              LOG.debug("Found SST node: {}", sstFile);
+              it.remove();
+            }
+          }
+
+          if (loadSet.size() == 0) {
+            break;
+          }
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+
+    // Just in case there are still nodes to be expected not loaded.
+    if (loadSet.size() > 0) {
+      LOG.warn("The following nodes are missing from the compaction log: {}. "
+          + "Possibly because those a newly flushed SSTs that haven't gone "
+          + "though any compaction yet", loadSet);
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Load existing compaction log files to the in-memory DAG.
+   * This only needs to be done once during OM startup.
+   */
+  public synchronized void loadAllCompactionLogs() {
+    if (compactionLogDir == null) {
+      throw new RuntimeException("Compaction log directory must be set first");
+    }
+    reconstructionSnapshotGeneration = 0L;
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase().endsWith(".log"))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+          readCompactionLogToDAG(logPath.toString());
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+  }
+
+  /**
+   * Get a list of SST files that differs between src and destination snapshots.
+   * <p>
+   * Expected input: src is a snapshot taken AFTER the dest.
+   *
+   * @param src source snapshot
+   * @param dest destination snapshot
+   */
+  public synchronized List<String> getSSTDiffList(Snapshot src, Snapshot dest) {
+
+    LOG.debug("src '{}' -> dest '{}'", src.dbPath, dest.dbPath);
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
 
     HashSet<String> fwdDAGSameFiles = new HashSet<>();
     HashSet<String> fwdDAGDifferentFiles = new HashSet<>();
 
-    LOG.warn("Doing forward diff between source and destination " +
-        "Snapshots:" + src.dbPath + ", " + dest.dbPath);
-    realPrintSnapdiffSSTFiles(src, dest, srcSnapFiles, destSnapFiles,
-        compactionDAGFwd,
-        fwdDAGSameFiles,
-        fwdDAGDifferentFiles);
+    LOG.debug("Doing forward diff between src and dest snapshots: " +
+        src.dbPath + " to " + dest.dbPath);
+    internalGetSSTDiffList(src, dest, srcSnapFiles, destSnapFiles,
+        compactionDAGFwd, fwdDAGSameFiles, fwdDAGDifferentFiles);
 
-    LOG.warn("Overall Summary \n" +
-            "Doing Overall diff between source and destination Snapshots:" +
-        src.dbPath + ", " + dest.dbPath);
-    System.out.print("fwd DAG Same files :");
-    for (String file : fwdDAGSameFiles) {
-      System.out.print(file + ", ");
-    }
-    LOG.warn("");
-    System.out.print("\nFwd DAG Different files :");
-    for (String file : fwdDAGDifferentFiles) {
-      CompactionNode n = compactionNodeTable.get(file);
-      System.out.print(file + ", ");
+    List<String> res = new ArrayList<>();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Result of diff from src '" + src.dbPath + "' to dest '" +
+          dest.dbPath + "':");
+      StringBuilder logSB = new StringBuilder();
+
+      logSB.append("Fwd DAG same SST files:      ");
+      for (String file : fwdDAGSameFiles) {
+        logSB.append(file).append(" ");
+      }
+      LOG.debug(logSB.toString());
+
+      logSB.setLength(0);
+      logSB.append("Fwd DAG different SST files: ");
+      for (String file : fwdDAGDifferentFiles) {
+        logSB.append(file).append(" ");
+        res.add(file);
+      }
+      LOG.debug(logSB.toString());
+
+    } else {
+      res.addAll(fwdDAGDifferentFiles);
     }
-    LOG.warn("");
+
+    return res;
   }
 
   @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-  public synchronized void realPrintSnapdiffSSTFiles(
+  public synchronized void internalGetSSTDiffList(
       Snapshot src, Snapshot dest,
       HashSet<String> srcSnapFiles,
       HashSet<String> destSnapFiles,
       MutableGraph<CompactionNode> mutableGraph,
       HashSet<String> sameFiles, HashSet<String> differentFiles) {
 
-
     for (String fileName : srcSnapFiles) {
       if (destSnapFiles.contains(fileName)) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + " and Dest " +
-            "Snapshot" + dest.dbPath + " Contain Same file " + fileName);
+        LOG.debug("Source '{}' and destination '{}' share the same SST '{}'",
+            src.dbPath, dest.dbPath, fileName);
         sameFiles.add(fileName);
         continue;
       }
       CompactionNode infileNode =
           compactionNodeTable.get(Paths.get(fileName).getFileName().toString());
       if (infileNode == null) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + "File " + fileName + "was " +
-            "never compacted");
+        LOG.debug("Src " + src.dbPath + " File " + fileName +
+            " was never compacted");
         differentFiles.add(fileName);
         continue;
       }
-      System.out.print(" Expandin File:" + fileName + ":\n");
-      Set<CompactionNode> nextLevel = new HashSet<>();
-      nextLevel.add(infileNode);
+      LOG.debug("Expanding SST file: " + fileName);
       Set<CompactionNode> currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
-      nextLevel = new HashSet<>();
+      currentLevel.add(infileNode);
+      Set<CompactionNode> nextLevel = new HashSet<>();

Review Comment:
   If you move `nextLevel` deceleration and initialization  inside while loop (line # 850), you won't have to reset it at line # 887.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993956025


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -230,196 +330,154 @@ public void createCheckPoint(String dbPathArg, String cpPathArg,
     }
   }
 
+  /**
+   * Append (then flush) to the current compaction log file path.
+   */
+  public void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogFilename == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Current compaction log filename is not set. "
+          + "Please check initialization.");
+      return;
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogFilename),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogFilename, e);
+    }
+  }
+
+  /**
+   * This takes Options.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  /**
+   * This takes DBOptions.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
   public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (db) {
+
+          if (compactionJobInfo.inputFiles().size() == 0) {
+            LOG.error("Compaction input files list is empty?");
+            return;
+          }
+
+          // Create hardlink backups for the SST files that are going
+          // to be deleted after this RDB compaction.
+          for (String file : compactionJobInfo.inputFiles()) {
+            LOG.info("Hardlinking {}", file);
+            String saveLinkFileName =
+                saveCompactedFilePath + new File(file).getName();
+            Path link = Paths.get(saveLinkFileName);
+            Path srcFile = Paths.get(file);
+            try {
+              Files.createLink(link, srcFile);
+            } catch (IOException e) {
+              LOG.error("Exception in creating hardlink for {}", file);
+              e.printStackTrace();

Review Comment:
   To me, this error is serious enough to throw a runtime exception.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1000032716


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -145,16 +235,29 @@ public RocksDBCheckpointDiffer(String dbPath,
     //  mapping.
   }
 
+  /**
+   * Helper function that recursively deletes the dir. TODO: REMOVE
+   */
+  boolean deleteDirectory(File directoryToBeDeleted) {
+    File[] allContents = directoryToBeDeleted.listFiles();
+    if (allContents != null) {
+      for (File file : allContents) {
+        deleteDirectory(file);
+      }
+    }
+    return directoryToBeDeleted.delete();
+  }
+
   // Node in the DAG to represent an SST file
   private class CompactionNode {
-    public String fileName;   // Name of the SST file
-    public String snapshotId; // The last snapshot that was created before this
+    private String fileName;   // Name of the SST file
+    private String snapshotId; // The last snapshot that was created before this
     // node came into existance;
-    public long snapshotGeneration;
-    public long totalNumberOfKeys;
-    public long cumulativeKeysReverseTraversal;
+    private long snapshotGeneration;
+    private long totalNumberOfKeys;
+    private long cumulativeKeysReverseTraversal;
 
-    CompactionNode (String f, String sid, long numKeys, long compactionGen) {
+    CompactionNode(String f, String sid, long numKeys, long compactionGen) {
       fileName = f;
       snapshotId = sid;
       snapshotGeneration = lastSnapshotCounter;

Review Comment:
   You are right. Should be fix by e76a24eddefcc44ab429f2be3245e0ec252c7d5a now.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993951742


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -114,9 +113,82 @@ public class RocksDBCheckpointDiffer {
   private static final long UNKNOWN_COMPACTION_GEN = 0;
   private long currentCompactionGen = 0;
 
-  // Something to track all the snapshots created so far.
+  // Something to track all the snapshots created so far. TODO: REMOVE
   private Snapshot[] allSnapshots;
 
+  private String compactionLogParentDir = null;
+  private String compactionLogDir = null;
+
+  // Name of the directory to hold compaction logs (under parent dir)
+  private static final String COMPACTION_LOG_DIR = "compaction-log/";
+
+  // For DB compaction SST DAG persistence and reconstruction
+  // Should be initialized to the latest sequence number
+  private volatile String currentCompactionLogFilename = null;
+
+  private static final String COMPACTION_LOG_FILENAME_SUFFIX = ".log";
+
+  public void setCompactionLogParentDir(String parentDir) {
+    this.compactionLogParentDir = parentDir;
+
+    // Append /
+    if (!compactionLogParentDir.endsWith("/")) {
+      compactionLogParentDir += "/";
+    }
+
+    File pDir = new File(compactionLogParentDir);
+    if (!pDir.exists()) {
+      if (!pDir.mkdir()) {
+        LOG.error("Error creating compaction log parent dir.");
+        return;
+      }
+    }
+
+    compactionLogDir =
+        Paths.get(compactionLogParentDir, COMPACTION_LOG_DIR).toString();
+    File clDir = new File(compactionLogDir);
+    if (!clDir.mkdir()) {
+      LOG.error("Error creating compaction log dir.");
+      return;
+    }
+
+    // TODO: Write a README there explaining what the dir is for
+  }
+
+  private static final int LONG_MAX_STRLEN =
+      String.valueOf(Long.MAX_VALUE).length();
+
+  public void setCompactionLogFilenameBySeqNum(long latestSequenceId) {
+    String latestSequenceIdStr = String.valueOf(latestSequenceId);
+
+    if (latestSequenceIdStr.length() < LONG_MAX_STRLEN) {
+      // Pad zeroes to the left for sequential listing later
+      latestSequenceIdStr =
+          StringUtils.leftPad(latestSequenceIdStr, LONG_MAX_STRLEN, "0");
+    }
+
+    this.currentCompactionLogFilename = compactionLogParentDir +
+        COMPACTION_LOG_DIR + latestSequenceIdStr +
+        COMPACTION_LOG_FILENAME_SUFFIX;
+
+    // Create empty file when if it doesn't exist.
+    // Detect any write permission issue (if any) early.
+    File clFile = new File(currentCompactionLogFilename);
+    if (clFile.exists()) {
+      LOG.warn("Compaction log exists: {}. Will append",

Review Comment:
   It seems like things will break, if appendToCurrentCompactionLog() gets called right here between the if statement and the else statement in another thread



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993140147


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,13 +529,151 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   * TODO: Drop synchronized? and make this thread safe?
+   */
+  private synchronized void processCompactionLogLine(String line) {
+    LOG.info("Processing line: {}", line);
+
+    // Skip comments
+    if (line.startsWith("#")) {
+      LOG.info("Skipped comment.");
+      return;
+    }
+
+    if (sstTokensRead == null) {
+      // Store the tokens in the first line
+      sstTokensRead = line.split("\t");
+      LOG.info("Length of inputFiles = {}", sstTokensRead.length);
+      if (sstTokensRead.length == 0) {
+        // Sanity check. inputFiles should never be empty. outputFiles could.
+        throw new RuntimeException(
+            "Compaction inputFiles list is empty. File is corrupted?");
+      }
+    } else {
+      final String[] outputFilesRead = line.split("\t");
+      LOG.info("Length of outputFiles = {}", outputFilesRead.length);
+
+      // Populate the compaction DAG
+      populateCompactionDAG(sstTokensRead, outputFilesRead);
+
+      // Reset inputFilesRead to null so
+      sstTokensRead = null;
+    }
+  }
+
+  private String[] sstTokensRead;
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   * <p>
+   * DO NOT use this function in another context without understanding what it
+   * does, due to the state preserved between calls (in sstTokensRead).
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> stream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      assert (sstTokensRead == null);
+      stream.forEach(this::processCompactionLogLine);
+      if (sstTokensRead != null) {
+        // Sanity check. Temp variable must be null after parsing.
+        // Otherwise it means the compaction log is corrupted.
+        throw new RuntimeException("Missing output files line. Corrupted?");
+      }
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Read compaction log until all dest (and src) db checkpoint SST
+   * nodes show up in the graph, or when it reaches the end of the log.
+   */
+  private boolean loadCompactionDAGBySSTSet(HashSet<String> sstSet) {
+
+    // Make a copy of the sstSet
+    HashSet<String> loadSet = new HashSet<>(sstSet);
+
+    // Check if all the nodes in the provided SST set is already loaded in DAG
+    for (String sstFile : sstSet) {
+      if (compactionNodeTable.containsKey(sstFile)) {
+        loadSet.remove(sstFile);
+      }
+    }
+
+    if (loadSet.size() == 0) {
+      // All expected nodes in the sstSet are already there,
+      //  no need to read any compaction log
+      return true;
+    }
+
+    // Otherwise, load compaction logs in order until all nodes are present in
+    //  the DAG.
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase().endsWith(".log"))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+
+          // TODO: Potential optimization: stop reading as soon as all nodes are
+          //  there. Currently it loads an entire file at a time.
+          readCompactionLogToDAG(logPath.toString());
+
+          for (Iterator<String> it = loadSet.iterator(); it.hasNext();) {
+            String sstFile = it.next();
+            if (compactionNodeTable.containsKey(sstFile)) {
+              LOG.debug("Found SST node: {}", sstFile);
+              it.remove();
+            }
+          }
+
+          if (loadSet.size() == 0) {
+            break;
+          }
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+
+    // Just in case there are still nodes to be expected not loaded.
+    if (loadSet.size() > 0) {
+      LOG.warn("The following nodes are missing from the compaction log: {}. "
+          + "Possibly because those a newly flushed SSTs that haven't gone "
+          + "though any compaction yet", loadSet);
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Given the src and destination Snapshots, it prints a Diff list.
+   *
+   * Expected input: src is a checkpoint taken AFTER dest checkpoint.
+   *
+   * @param src
+   * @param dest
+   * @throws RocksDBException
+   */
+  private synchronized List<String> printSnapdiffSSTFiles(
+      Snapshot src, Snapshot dest) {
+
+    LOG.warn("src db checkpoint: {}", src.dbPath);  // from
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
+    LOG.warn("dest db checkpoint: {}", dest.dbPath);  //to
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
+    System.out.println();
+
+    // Read compaction log until all dest (and src) db checkpoint SST
+    // nodes show up in the graph
+//    loadCompactionDAGBySSTSet(destSnapFiles);

Review Comment:
   Yes this is intentional because in reality it is redundant. When loading `srcSnapFiles`, `destSnapFiles` should have been traversed already under the current assumption that:
   
   > Expected input: src is a checkpoint taken AFTER dest checkpoint



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993954747


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -145,16 +235,29 @@ public RocksDBCheckpointDiffer(String dbPath,
     //  mapping.
   }
 
+  /**
+   * Helper function that recursively deletes the dir. TODO: REMOVE
+   */
+  boolean deleteDirectory(File directoryToBeDeleted) {
+    File[] allContents = directoryToBeDeleted.listFiles();
+    if (allContents != null) {
+      for (File file : allContents) {
+        deleteDirectory(file);
+      }
+    }
+    return directoryToBeDeleted.delete();
+  }
+
   // Node in the DAG to represent an SST file
   private class CompactionNode {
-    public String fileName;   // Name of the SST file
-    public String snapshotId; // The last snapshot that was created before this
+    private String fileName;   // Name of the SST file
+    private String snapshotId; // The last snapshot that was created before this
     // node came into existance;
-    public long snapshotGeneration;
-    public long totalNumberOfKeys;
-    public long cumulativeKeysReverseTraversal;
+    private long snapshotGeneration;
+    private long totalNumberOfKeys;
+    private long cumulativeKeysReverseTraversal;
 
-    CompactionNode (String f, String sid, long numKeys, long compactionGen) {
+    CompactionNode(String f, String sid, long numKeys, long compactionGen) {
       fileName = f;
       snapshotId = sid;
       snapshotGeneration = lastSnapshotCounter;

Review Comment:
   It seems  incorrect to me to have the snapshotGeneration be set by the lastSnapshotCounter at the time the node is created.  To me it should be the value of the lastSnapshotCounter at the time the compaction log file was created, (in other words it probably should be stored and read from the compaction log.)
   



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993959701


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -145,16 +235,29 @@ public RocksDBCheckpointDiffer(String dbPath,
     //  mapping.
   }
 
+  /**
+   * Helper function that recursively deletes the dir. TODO: REMOVE
+   */
+  boolean deleteDirectory(File directoryToBeDeleted) {
+    File[] allContents = directoryToBeDeleted.listFiles();
+    if (allContents != null) {
+      for (File file : allContents) {
+        deleteDirectory(file);
+      }
+    }
+    return directoryToBeDeleted.delete();
+  }
+
   // Node in the DAG to represent an SST file
   private class CompactionNode {
-    public String fileName;   // Name of the SST file
-    public String snapshotId; // The last snapshot that was created before this
+    private String fileName;   // Name of the SST file
+    private String snapshotId; // The last snapshot that was created before this
     // node came into existance;
-    public long snapshotGeneration;
-    public long totalNumberOfKeys;
-    public long cumulativeKeysReverseTraversal;
+    private long snapshotGeneration;
+    private long totalNumberOfKeys;
+    private long cumulativeKeysReverseTraversal;
 
-    CompactionNode (String f, String sid, long numKeys, long compactionGen) {
+    CompactionNode(String f, String sid, long numKeys, long compactionGen) {

Review Comment:
   compactionGen parameter isn't used



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] prashantpogde commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1004944027


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -230,196 +330,154 @@ public void createCheckPoint(String dbPathArg, String cpPathArg,
     }
   }
 
+  /**
+   * Append (then flush) to the current compaction log file path.
+   */
+  public void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogFilename == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Current compaction log filename is not set. "
+          + "Please check initialization.");
+      return;
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogFilename),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogFilename, e);
+    }
+  }
+
+  /**
+   * This takes Options.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  /**
+   * This takes DBOptions.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
   public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (db) {

Review Comment:
   If we don't synchronize here, the DAG can get corrupted. RocksDB  could have multiple compactions happening concurrently  in independent threads.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on PR #3824:
URL: https://github.com/apache/ozone/pull/3824#issuecomment-1299590741

   This PR is ready for another round of review.
   
   UT `TestRocksDBCheckpointDiffer` output for reference:
   
   ```
   2022-11-01 22:22:44,258 [main] INFO  rocksdiff.TestRocksDBCheckpointDiffer (TestRocksDBCheckpointDiffer.java:diffAllSnapshots(170)) - SST diff from 'rocksdb-cp-250000' to 'rocksdb-cp-1': [000013, 000022, 000011, 000018, 000015, 000020]
   2022-11-01 22:22:44,261 [main] INFO  rocksdiff.TestRocksDBCheckpointDiffer (TestRocksDBCheckpointDiffer.java:diffAllSnapshots(170)) - SST diff from 'rocksdb-cp-250000' to 'rocksdb-cp-50000': [000013, 000022, 000018, 000015, 000020]
   2022-11-01 22:22:44,263 [main] INFO  rocksdiff.TestRocksDBCheckpointDiffer (TestRocksDBCheckpointDiffer.java:diffAllSnapshots(170)) - SST diff from 'rocksdb-cp-250000' to 'rocksdb-cp-99999': [000022, 000018, 000015, 000020]
   2022-11-01 22:22:44,265 [main] INFO  rocksdiff.TestRocksDBCheckpointDiffer (TestRocksDBCheckpointDiffer.java:diffAllSnapshots(170)) - SST diff from 'rocksdb-cp-250000' to 'rocksdb-cp-149998': [000022, 000018, 000020]
   2022-11-01 22:22:44,267 [main] INFO  rocksdiff.TestRocksDBCheckpointDiffer (TestRocksDBCheckpointDiffer.java:diffAllSnapshots(170)) - SST diff from 'rocksdb-cp-250000' to 'rocksdb-cp-199997': [000022, 000020]
   2022-11-01 22:22:44,269 [main] INFO  rocksdiff.TestRocksDBCheckpointDiffer (TestRocksDBCheckpointDiffer.java:diffAllSnapshots(170)) - SST diff from 'rocksdb-cp-250000' to 'rocksdb-cp-249996': [000022]
   2022-11-01 22:22:44,272 [main] INFO  rocksdiff.TestRocksDBCheckpointDiffer (TestRocksDBCheckpointDiffer.java:diffAllSnapshots(170)) - SST diff from 'rocksdb-cp-250000' to 'rocksdb-cp-250000': []
   ```


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1010744176


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java:
##########
@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.freon;
+
+import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.raftlog.RaftLog;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
+import picocli.CommandLine;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_S3_VOLUME_NAME_DEFAULT;
+
+/**
+ * Tests Freon, with MiniOzoneCluster.
+ */
+public class TestOMSnapshotDAG {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOMSnapshotDAG.class);
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+  private static ObjectStore store;
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting OZONE_ENABLED = true
+   *
+   */
+  @BeforeAll
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    DatanodeRatisServerConfig ratisServerConfig =
+        conf.getObject(DatanodeRatisServerConfig.class);
+    ratisServerConfig.setRequestTimeOut(Duration.ofSeconds(3));
+    ratisServerConfig.setWatchTimeOut(Duration.ofSeconds(3));
+    conf.setFromObject(ratisServerConfig);
+
+    RatisClientConfig.RaftConfig raftClientConfig =
+        conf.getObject(RatisClientConfig.RaftConfig.class);
+    raftClientConfig.setRpcRequestTimeout(Duration.ofSeconds(3));
+    raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(3));
+    conf.setFromObject(raftClientConfig);
+
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(5).build();
+    cluster.waitForClusterToBeReady();
+
+    store = cluster.getClient().getObjectStore();
+
+    GenericTestUtils.setLogLevel(RaftLog.LOG, Level.INFO);
+    GenericTestUtils.setLogLevel(RaftServer.LOG, Level.INFO);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterAll
+  public static void shutdown() {
+    if (cluster != null) {
+      LOG.warn("Waiting for an extra 10 seconds before shutting down...");
+      try {
+        Thread.sleep(10000);
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  void testZeroSizeKey() throws IOException {
+
+    RandomKeyGenerator randomKeyGenerator =
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "6000",
+        "--num-of-threads", "1",
+        "--key-size", "0",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
+    Assert.assertEquals(6000L, randomKeyGenerator.getNumberOfKeysAdded());
+    Assert.assertEquals(6000L,
+        randomKeyGenerator.getSuccessfulValidationCount());
+
+    List<OmVolumeArgs> volList = cluster.getOzoneManager()
+        .listAllVolumes("", "", 10);
+    System.out.println(volList);
+    final String volumeName = volList.stream().filter(e ->
+        !e.getVolume().equals(OZONE_S3_VOLUME_NAME_DEFAULT))  // Ignore s3v vol
+        .collect(Collectors.toList()).get(0).getVolume();
+    List<OmBucketInfo> bucketList =
+        cluster.getOzoneManager().listBuckets(volumeName, "", "", 10);
+    System.out.println(bucketList);
+    final String bucketName = bucketList.get(0).getBucketName();
+
+    // Create snapshot
+    String resp = store.createSnapshot(volumeName, bucketName, "snap1");
+    System.out.println(resp);
+
+    final OzoneVolume volume = store.getVolume(volumeName);
+    final OzoneBucket bucket = volume.getBucket(bucketName);
+
+    for (int i = 0; i < 6000; i++) {
+      bucket.createKey("b_" + i, 0).close();
+    }
+
+    resp = store.createSnapshot(volumeName, bucketName, "snap3");
+    System.out.println(resp);
+  }
+

Review Comment:
   Sounds good. I would do this in a follow-up jira together with some new debug options in RDBStore.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993954747


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -145,16 +235,29 @@ public RocksDBCheckpointDiffer(String dbPath,
     //  mapping.
   }
 
+  /**
+   * Helper function that recursively deletes the dir. TODO: REMOVE
+   */
+  boolean deleteDirectory(File directoryToBeDeleted) {
+    File[] allContents = directoryToBeDeleted.listFiles();
+    if (allContents != null) {
+      for (File file : allContents) {
+        deleteDirectory(file);
+      }
+    }
+    return directoryToBeDeleted.delete();
+  }
+
   // Node in the DAG to represent an SST file
   private class CompactionNode {
-    public String fileName;   // Name of the SST file
-    public String snapshotId; // The last snapshot that was created before this
+    private String fileName;   // Name of the SST file
+    private String snapshotId; // The last snapshot that was created before this
     // node came into existance;
-    public long snapshotGeneration;
-    public long totalNumberOfKeys;
-    public long cumulativeKeysReverseTraversal;
+    private long snapshotGeneration;
+    private long totalNumberOfKeys;
+    private long cumulativeKeysReverseTraversal;
 
-    CompactionNode (String f, String sid, long numKeys, long compactionGen) {
+    CompactionNode(String f, String sid, long numKeys, long compactionGen) {
       fileName = f;
       snapshotId = sid;
       snapshotGeneration = lastSnapshotCounter;

Review Comment:
   It seems to incorrect to me to have the snapshotGeneration be set by the lastSnapshotCounter at the time the node is created.  To me it should be the value of the lastSnapshotCounter at the time the compaction log file was created, (in other words it probably should be stored and read from the compaction log.)
   



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993958846


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -131,6 +132,22 @@ public static DBCheckpoint createOmSnapshotCheckpoint(
       OMMetadataManager omMetadataManager, SnapshotInfo snapshotInfo)
       throws IOException {
     RDBStore store = (RDBStore) omMetadataManager.getStore();
+
+    final long dbLatestSequenceNumber =
+        ((RDBStore) omMetadataManager.getStore()).getDb()
+            .getLatestSequenceNumber();
+
+    final RocksDBCheckpointDiffer checkpointDiffer =
+        omMetadataManager.getStore().getRocksDBCheckpointDiffer();
+
+    // Set compaction log filename to the latest DB sequence number
+    // right before taking the RocksDB checkpoint
+    //
+    // Note it doesn't matter if sequence number hasn't increased (even though
+    // it shouldn't happen), since the writer always appends the file.
+    checkpointDiffer.setCompactionLogParentDir(store.getSnapshotsParentDir());
+    checkpointDiffer.setCompactionLogFilenameBySeqNum(dbLatestSequenceNumber);
+

Review Comment:
   does the lastSnapshotCounter need to incremented here?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r999286854


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -114,9 +113,82 @@ public class RocksDBCheckpointDiffer {
   private static final long UNKNOWN_COMPACTION_GEN = 0;
   private long currentCompactionGen = 0;
 
-  // Something to track all the snapshots created so far.
+  // Something to track all the snapshots created so far. TODO: REMOVE
   private Snapshot[] allSnapshots;
 
+  private String compactionLogParentDir = null;
+  private String compactionLogDir = null;
+
+  // Name of the directory to hold compaction logs (under parent dir)
+  private static final String COMPACTION_LOG_DIR = "compaction-log/";
+
+  // For DB compaction SST DAG persistence and reconstruction
+  // Should be initialized to the latest sequence number
+  private volatile String currentCompactionLogFilename = null;
+
+  private static final String COMPACTION_LOG_FILENAME_SUFFIX = ".log";
+
+  public void setCompactionLogParentDir(String parentDir) {
+    this.compactionLogParentDir = parentDir;
+
+    // Append /
+    if (!compactionLogParentDir.endsWith("/")) {
+      compactionLogParentDir += "/";
+    }
+
+    File pDir = new File(compactionLogParentDir);
+    if (!pDir.exists()) {
+      if (!pDir.mkdir()) {
+        LOG.error("Error creating compaction log parent dir.");
+        return;
+      }
+    }
+
+    compactionLogDir =
+        Paths.get(compactionLogParentDir, COMPACTION_LOG_DIR).toString();
+    File clDir = new File(compactionLogDir);
+    if (!clDir.mkdir()) {
+      LOG.error("Error creating compaction log dir.");
+      return;
+    }
+
+    // TODO: Write a README there explaining what the dir is for
+  }
+
+  private static final int LONG_MAX_STRLEN =
+      String.valueOf(Long.MAX_VALUE).length();
+
+  public void setCompactionLogFilenameBySeqNum(long latestSequenceId) {
+    String latestSequenceIdStr = String.valueOf(latestSequenceId);
+
+    if (latestSequenceIdStr.length() < LONG_MAX_STRLEN) {
+      // Pad zeroes to the left for sequential listing later
+      latestSequenceIdStr =
+          StringUtils.leftPad(latestSequenceIdStr, LONG_MAX_STRLEN, "0");
+    }
+
+    this.currentCompactionLogFilename = compactionLogParentDir +
+        COMPACTION_LOG_DIR + latestSequenceIdStr +
+        COMPACTION_LOG_FILENAME_SUFFIX;
+
+    // Create empty file when if it doesn't exist.
+    // Detect any write permission issue (if any) early.
+    File clFile = new File(currentCompactionLogFilename);
+    if (clFile.exists()) {
+      LOG.warn("Compaction log exists: {}. Will append",

Review Comment:
   Good point. I am now using a local variable first when doing the existence check. And I have also added `synchroized` keyword to `appendToCurrentCompactionLog()` and is calling that in `setCurrentCompactionLog()` instead.



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -114,9 +113,82 @@ public class RocksDBCheckpointDiffer {
   private static final long UNKNOWN_COMPACTION_GEN = 0;
   private long currentCompactionGen = 0;
 
-  // Something to track all the snapshots created so far.
+  // Something to track all the snapshots created so far. TODO: REMOVE
   private Snapshot[] allSnapshots;
 
+  private String compactionLogParentDir = null;
+  private String compactionLogDir = null;
+
+  // Name of the directory to hold compaction logs (under parent dir)
+  private static final String COMPACTION_LOG_DIR = "compaction-log/";
+
+  // For DB compaction SST DAG persistence and reconstruction
+  // Should be initialized to the latest sequence number
+  private volatile String currentCompactionLogFilename = null;
+
+  private static final String COMPACTION_LOG_FILENAME_SUFFIX = ".log";
+
+  public void setCompactionLogParentDir(String parentDir) {
+    this.compactionLogParentDir = parentDir;
+
+    // Append /
+    if (!compactionLogParentDir.endsWith("/")) {
+      compactionLogParentDir += "/";
+    }
+
+    File pDir = new File(compactionLogParentDir);
+    if (!pDir.exists()) {
+      if (!pDir.mkdir()) {
+        LOG.error("Error creating compaction log parent dir.");
+        return;
+      }
+    }
+
+    compactionLogDir =
+        Paths.get(compactionLogParentDir, COMPACTION_LOG_DIR).toString();
+    File clDir = new File(compactionLogDir);
+    if (!clDir.mkdir()) {
+      LOG.error("Error creating compaction log dir.");
+      return;
+    }
+
+    // TODO: Write a README there explaining what the dir is for
+  }
+
+  private static final int LONG_MAX_STRLEN =
+      String.valueOf(Long.MAX_VALUE).length();
+
+  public void setCompactionLogFilenameBySeqNum(long latestSequenceId) {

Review Comment:
   Done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r999306074


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,13 +529,151 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   * TODO: Drop synchronized? and make this thread safe?
+   */
+  private synchronized void processCompactionLogLine(String line) {
+    LOG.info("Processing line: {}", line);
+
+    // Skip comments
+    if (line.startsWith("#")) {
+      LOG.info("Skipped comment.");
+      return;
+    }
+
+    if (sstTokensRead == null) {
+      // Store the tokens in the first line
+      sstTokensRead = line.split("\t");
+      LOG.info("Length of inputFiles = {}", sstTokensRead.length);
+      if (sstTokensRead.length == 0) {
+        // Sanity check. inputFiles should never be empty. outputFiles could.
+        throw new RuntimeException(
+            "Compaction inputFiles list is empty. File is corrupted?");
+      }
+    } else {
+      final String[] outputFilesRead = line.split("\t");
+      LOG.info("Length of outputFiles = {}", outputFilesRead.length);
+
+      // Populate the compaction DAG
+      populateCompactionDAG(sstTokensRead, outputFilesRead);
+
+      // Reset inputFilesRead to null so
+      sstTokensRead = null;
+    }
+  }
+
+  private String[] sstTokensRead;
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   * <p>
+   * DO NOT use this function in another context without understanding what it
+   * does, due to the state preserved between calls (in sstTokensRead).
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> stream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      assert (sstTokensRead == null);
+      stream.forEach(this::processCompactionLogLine);
+      if (sstTokensRead != null) {
+        // Sanity check. Temp variable must be null after parsing.
+        // Otherwise it means the compaction log is corrupted.
+        throw new RuntimeException("Missing output files line. Corrupted?");
+      }
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Read compaction log until all dest (and src) db checkpoint SST
+   * nodes show up in the graph, or when it reaches the end of the log.
+   */
+  private boolean loadCompactionDAGBySSTSet(HashSet<String> sstSet) {
+
+    // Make a copy of the sstSet
+    HashSet<String> loadSet = new HashSet<>(sstSet);
+
+    // Check if all the nodes in the provided SST set is already loaded in DAG
+    for (String sstFile : sstSet) {
+      if (compactionNodeTable.containsKey(sstFile)) {
+        loadSet.remove(sstFile);
+      }
+    }
+
+    if (loadSet.size() == 0) {
+      // All expected nodes in the sstSet are already there,
+      //  no need to read any compaction log
+      return true;
+    }
+
+    // Otherwise, load compaction logs in order until all nodes are present in
+    //  the DAG.
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase().endsWith(".log"))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+
+          // TODO: Potential optimization: stop reading as soon as all nodes are
+          //  there. Currently it loads an entire file at a time.
+          readCompactionLogToDAG(logPath.toString());
+
+          for (Iterator<String> it = loadSet.iterator(); it.hasNext();) {
+            String sstFile = it.next();
+            if (compactionNodeTable.containsKey(sstFile)) {
+              LOG.debug("Found SST node: {}", sstFile);
+              it.remove();
+            }
+          }
+
+          if (loadSet.size() == 0) {
+            break;
+          }
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+
+    // Just in case there are still nodes to be expected not loaded.
+    if (loadSet.size() > 0) {
+      LOG.warn("The following nodes are missing from the compaction log: {}. "
+          + "Possibly because those a newly flushed SSTs that haven't gone "
+          + "though any compaction yet", loadSet);
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Given the src and destination Snapshots, it prints a Diff list.
+   *
+   * Expected input: src is a checkpoint taken AFTER dest checkpoint.
+   *
+   * @param src
+   * @param dest
+   * @throws RocksDBException
+   */
+  private synchronized List<String> printSnapdiffSSTFiles(
+      Snapshot src, Snapshot dest) {
+
+    LOG.warn("src db checkpoint: {}", src.dbPath);  // from
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
+    LOG.warn("dest db checkpoint: {}", dest.dbPath);  //to
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
+    System.out.println();
+
+    // Read compaction log until all dest (and src) db checkpoint SST
+    // nodes show up in the graph
+//    loadCompactionDAGBySSTSet(destSnapFiles);

Review Comment:
   Done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1000032300


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -145,16 +235,29 @@ public RocksDBCheckpointDiffer(String dbPath,
     //  mapping.
   }
 
+  /**
+   * Helper function that recursively deletes the dir. TODO: REMOVE
+   */
+  boolean deleteDirectory(File directoryToBeDeleted) {
+    File[] allContents = directoryToBeDeleted.listFiles();
+    if (allContents != null) {
+      for (File file : allContents) {
+        deleteDirectory(file);
+      }
+    }
+    return directoryToBeDeleted.delete();
+  }
+
   // Node in the DAG to represent an SST file
   private class CompactionNode {
-    public String fileName;   // Name of the SST file
-    public String snapshotId; // The last snapshot that was created before this
+    private String fileName;   // Name of the SST file
+    private String snapshotId; // The last snapshot that was created before this
     // node came into existance;
-    public long snapshotGeneration;
-    public long totalNumberOfKeys;
-    public long cumulativeKeysReverseTraversal;
+    private long snapshotGeneration;
+    private long totalNumberOfKeys;
+    private long cumulativeKeysReverseTraversal;
 
-    CompactionNode (String f, String sid, long numKeys, long compactionGen) {
+    CompactionNode(String f, String sid, long numKeys, long compactionGen) {

Review Comment:
   No it wasn't used.
   
   Now in the latest commit it has been repurposed to pass in the sequence number as snapshot generation.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012542936


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -215,390 +288,511 @@ public static void addDebugLevel(Integer level) {
     DEBUG_LEVEL.add(level);
   }
 
-  // Flushes the WAL and Creates a RocksDB checkpoint
-  @SuppressWarnings({"NM_METHOD_NAMING_CONVENTION"})
-  public void createCheckPoint(String dbPathArg, String cpPathArg,
-                               RocksDB rocksDB) {
-    LOG.warn("Creating Checkpoint for RocksDB instance : " +
-        dbPathArg + "in a CheckPoint Location" + cpPathArg);
-    try {
-      rocksDB.flush(new FlushOptions());
-      Checkpoint cp = Checkpoint.create(rocksDB);
-      cp.createCheckpoint(cpPathArg);
-    } catch (RocksDBException e) {
-      throw new RuntimeException(e.getMessage());
+  /**
+   * Append (then flush) to the current compaction log file path.
+   * Note: This does NOT automatically append newline to the log.
+   */
+  private synchronized void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogPath == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Compaction log path is not set. "
+          + "Please check initialization.");
+      throw new RuntimeException("Compaction log path not set");
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogPath),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogPath, e);
     }
   }
 
-  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
-      throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
-            }
-          }
-        };
+  /**
+   * Append a sequence number to the compaction log (roughly) when an Ozone
+   * snapshot (RDB checkpoint) is taken.
+   * @param sequenceNum RDB sequence number
+   */
+  public void appendSequenceNumberToCompactionLog(long sequenceNum) {
+    final String line = COMPACTION_LOG_SEQNUM_LINE_PREFIX + sequenceNum + "\n";
+    appendToCurrentCompactionLog(line);
+  }
 
-    list.add(onCompactionCompletedListener);
+  /**
+   * Takes {@link org.rocksdb.Options}.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
     rocksOptions.setListeners(list);
   }
 
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
 
+  /**
+   * Takes {@link org.rocksdb.DBOptions}.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
 
-  public void setRocksDBForCompactionTracking(Options rocksOptions)
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      Options rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db,final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file);
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix,
-                      numKeys, currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix, numKeys,
-                        UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null,
-                    compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().size() == 0) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          // Create hardlink backups for the SST files that are going
+          // to be deleted after this RDB compaction.
+          for (String file : compactionJobInfo.inputFiles()) {
+            LOG.debug("Creating hard link for '{}'", file);
+            String saveLinkFileName =
+                sstBackupDir + new File(file).getName();
+            Path link = Paths.get(saveLinkFileName);
+            Path srcFile = Paths.get(file);
+            try {
+              Files.createLink(link, srcFile);
+            } catch (IOException e) {
+              LOG.error("Exception in creating hard link for {}", file);
+              throw new RuntimeException("Failed to create hard link", e);
             }
           }
-        };
 
-    list.add(onCompactionCompletedListener);
-    rocksOptions.setListeners(list);
+        }
+      }
+    };
   }
 
-  public RocksDB getRocksDBInstanceWithCompactionTracking(String dbPath)
-      throws RocksDBException {
-    final Options opt = new Options().setCreateIfMissing(true)
-        .setCompressionType(CompressionType.NO_COMPRESSION);
-    opt.setMaxBytesForLevelMultiplier(2);
-    setRocksDBForCompactionTracking(opt);
-    return RocksDB.open(opt, dbPath);
+  private AbstractEventListener newCompactionCompletedListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionCompleted(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().isEmpty()) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          final StringBuilder sb = new StringBuilder();
+
+          if (LOG.isDebugEnabled()) {
+            // Print compaction reason for this entry in the log file
+            // e.g. kLevelL0FilesNum / kLevelMaxLevelSize.
+            sb.append(COMPACTION_LOG_COMMENT_LINE_PREFIX)
+                .append(compactionJobInfo.compactionReason())
+                .append('\n');
+          }
+
+          // Mark the beginning of a compaction log
+          sb.append(COMPACTION_LOG_ENTRY_LINE_PREFIX);
+
+          // Trim DB path, only keep the SST file name

Review Comment:
   The full path is highly redundant. IMO there is no need to persist the full path as the SST file can only either be in the SST backup dir or in active DB dir. Storing the full path easily doubles the in-memory map memory consumption compared to file name only.
   
   Also when this listener is triggered, the input / output SST file path is always under the active DB. But those SST files may well have been deleted from active DB instance later, and those SSTs are only kept under the backup dir.
   
   The logic you described is already implemented in `getSSTFileSummary()`:
   
   https://github.com/apache/ozone/pull/3824/files/bba897f83be886dc536b3fa81b0d437c17e00a14#diff-1d5c6c892a2d24d85ca002dafcb9f39341192baaa7b5323d2325b5fc7a238c7fR467-R472



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] hemantk-12 commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
hemantk-12 commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012193166


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -688,130 +868,139 @@ public synchronized void printMutableGraph(
       iter.remove();
       iter = topLevelNodes.iterator();
     }
-    LOG.warn("src snap:" + srcSnapId);
-    LOG.warn("dest snap:" + destSnapId);
+    LOG.debug("src snap: " + srcSnapId);
+    LOG.debug("dest snap: " + destSnapId);
     for (CompactionNode n : allNodes) {
-      LOG.warn("Files are :" + n.fileName);
+      LOG.debug("Files are: " + n.fileName);
     }
   }
 
+  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
+    return compactionDAGFwd;
+  }
 
-  public void createSnapshot(RocksDB rocksDB) throws InterruptedException {
-
-    LOG.warn("Current time is::" + System.currentTimeMillis());
-    long t1 = System.currentTimeMillis();
-
-    cpPath = cpPath + lastSnapshotCounter;
-    createCheckPoint(rocksDbPath, cpPath, rocksDB);
-    allSnapshots[lastSnapshotCounter] = new Snapshot(cpPath,
-    lastSnapshotPrefix, lastSnapshotCounter);
-
-    long t2 = System.currentTimeMillis();
-    LOG.warn("Current time is::" + t2);
-
-    LOG.warn("millisecond difference is ::" + (t2 - t1));
-   Thread.sleep(100);
-   ++lastSnapshotCounter;
-   lastSnapshotPrefix = "sid_" + lastSnapshotCounter;
-   LOG.warn("done :: 1");
+  public MutableGraph<CompactionNode> getCompactionReverseDAG() {
+    return compactionDAGReverse;
   }
 
+  /**
+   * Populate the compaction DAG with input and output SST files lists.
+   */
+  @SuppressFBWarnings({"AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION"})
+  private void populateCompactionDAG(List<String> inputFiles,
+      List<String> outputFiles, long seqNum) {
 
-  public void printAllSnapshots() throws InterruptedException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
-      }
-      LOG.warn("Snapshot id" + snap.snapshotID);
-      LOG.warn("Snapshot path" + snap.dbPath);
-      LOG.warn("Snapshot Generation" + snap.snapshotGeneration);
-      LOG.warn("");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Input {} -> Output {}", inputFiles, outputFiles);
     }
-  }
 
-  public void diffAllSnapshots() throws InterruptedException, RocksDBException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
+    for (String outfile : outputFiles) {
+      CompactionNode outfileNode = compactionNodeTable.get(outfile);
+      if (outfileNode == null) {
+        long numKeys = 0L;
+        try {
+          numKeys = getSSTFileSummary(outfile);
+        } catch (Exception e) {
+          LOG.warn("Exception in getSSTFileSummary: {}", e.getMessage());
+        }
+        outfileNode = new CompactionNode(outfile, null, numKeys, seqNum);
+        compactionDAGFwd.addNode(outfileNode);
+        compactionDAGReverse.addNode(outfileNode);
+        compactionNodeTable.put(outfile, outfileNode);
       }
-      printSnapdiffSSTFiles(allSnapshots[lastSnapshotCounter - 1], snap);
-    }
-  }
 
-  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
-    return compactionDAGFwd;
-  }
+      for (String infile : inputFiles) {
+        CompactionNode infileNode = compactionNodeTable.get(infile);

Review Comment:
   It looks better now. Thanks 👍 



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] hemantk-12 commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
hemantk-12 commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012192128


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -664,21 +844,21 @@ public synchronized void printMutableGraph(
         topLevelNodes.add(n);
       }
     }
-    Iterator iter = topLevelNodes.iterator();
+    Iterator<CompactionNode> iter = topLevelNodes.iterator();
     while (iter.hasNext()) {

Review Comment:
   Thanks for making the change.
   
   Sorry for so many refactoring comments.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] prashantpogde commented on pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on PR #3824:
URL: https://github.com/apache/ozone/pull/3824#issuecomment-1301545304

   LGTM for this PR. Additional set of changes that are highlighted in this PR can also go in the next PR. 


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1011189062


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -688,130 +868,139 @@ public synchronized void printMutableGraph(
       iter.remove();
       iter = topLevelNodes.iterator();
     }
-    LOG.warn("src snap:" + srcSnapId);
-    LOG.warn("dest snap:" + destSnapId);
+    LOG.debug("src snap: " + srcSnapId);
+    LOG.debug("dest snap: " + destSnapId);
     for (CompactionNode n : allNodes) {
-      LOG.warn("Files are :" + n.fileName);
+      LOG.debug("Files are: " + n.fileName);
     }
   }
 
+  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
+    return compactionDAGFwd;
+  }
 
-  public void createSnapshot(RocksDB rocksDB) throws InterruptedException {
-
-    LOG.warn("Current time is::" + System.currentTimeMillis());
-    long t1 = System.currentTimeMillis();
-
-    cpPath = cpPath + lastSnapshotCounter;
-    createCheckPoint(rocksDbPath, cpPath, rocksDB);
-    allSnapshots[lastSnapshotCounter] = new Snapshot(cpPath,
-    lastSnapshotPrefix, lastSnapshotCounter);
-
-    long t2 = System.currentTimeMillis();
-    LOG.warn("Current time is::" + t2);
-
-    LOG.warn("millisecond difference is ::" + (t2 - t1));
-   Thread.sleep(100);
-   ++lastSnapshotCounter;
-   lastSnapshotPrefix = "sid_" + lastSnapshotCounter;
-   LOG.warn("done :: 1");
+  public MutableGraph<CompactionNode> getCompactionReverseDAG() {
+    return compactionDAGReverse;
   }
 
+  /**
+   * Populate the compaction DAG with input and output SST files lists.
+   */
+  @SuppressFBWarnings({"AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION"})
+  private void populateCompactionDAG(List<String> inputFiles,
+      List<String> outputFiles, long seqNum) {
 
-  public void printAllSnapshots() throws InterruptedException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
-      }
-      LOG.warn("Snapshot id" + snap.snapshotID);
-      LOG.warn("Snapshot path" + snap.dbPath);
-      LOG.warn("Snapshot Generation" + snap.snapshotGeneration);
-      LOG.warn("");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Input {} -> Output {}", inputFiles, outputFiles);
     }
-  }
 
-  public void diffAllSnapshots() throws InterruptedException, RocksDBException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
+    for (String outfile : outputFiles) {
+      CompactionNode outfileNode = compactionNodeTable.get(outfile);
+      if (outfileNode == null) {
+        long numKeys = 0L;
+        try {
+          numKeys = getSSTFileSummary(outfile);
+        } catch (Exception e) {
+          LOG.warn("Exception in getSSTFileSummary: {}", e.getMessage());
+        }
+        outfileNode = new CompactionNode(outfile, null, numKeys, seqNum);
+        compactionDAGFwd.addNode(outfileNode);
+        compactionDAGReverse.addNode(outfileNode);
+        compactionNodeTable.put(outfile, outfileNode);
       }
-      printSnapdiffSSTFiles(allSnapshots[lastSnapshotCounter - 1], snap);
-    }
-  }
 
-  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
-    return compactionDAGFwd;
-  }
+      for (String infile : inputFiles) {
+        CompactionNode infileNode = compactionNodeTable.get(infile);

Review Comment:
   Done with `computeIfAbsent` and new helper method `addNodeToDAG`.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012320455


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -215,253 +304,252 @@ public static void addDebugLevel(Integer level) {
     DEBUG_LEVEL.add(level);
   }
 
-  // Flushes the WAL and Creates a RocksDB checkpoint
-  @SuppressWarnings({"NM_METHOD_NAMING_CONVENTION"})
-  public void createCheckPoint(String dbPathArg, String cpPathArg,
-                               RocksDB rocksDB) {
-    LOG.warn("Creating Checkpoint for RocksDB instance : " +
-        dbPathArg + "in a CheckPoint Location" + cpPathArg);
-    try {
-      rocksDB.flush(new FlushOptions());
-      Checkpoint cp = Checkpoint.create(rocksDB);
-      cp.createCheckpoint(cpPathArg);
-    } catch (RocksDBException e) {
-      throw new RuntimeException(e.getMessage());
+  /**
+   * Append (then flush) to the current compaction log file path.
+   * Note: This does NOT automatically append newline to the log.
+   */
+  private synchronized void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogPath == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Compaction log path is not set. "
+          + "Please check initialization.");
+      throw new RuntimeException("Compaction log path not set");
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogPath),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogPath, e);
     }
   }
 
-  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
-      throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
-            }
-          }
-        };
+  /**
+   * Append a sequence number to the compaction log (roughly) when an Ozone
+   * snapshot (RDB checkpoint) is taken.
+   * @param sequenceNum RDB sequence number
+   */
+  public void appendSequenceNumberToCompactionLog(long sequenceNum) {
+    final String line = COMPACTION_LOG_SEQNUM_LINE_PREFIX + sequenceNum + "\n";
+    appendToCurrentCompactionLog(line);
+  }
 
-    list.add(onCompactionCompletedListener);
+  /**
+   * Takes {@link org.rocksdb.Options}.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
     rocksOptions.setListeners(list);
   }
 
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
 
+  /**
+   * Takes {@link org.rocksdb.DBOptions}.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
 
-  public void setRocksDBForCompactionTracking(Options rocksOptions)
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      Options rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db,final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file);
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix,
-                      numKeys, currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix, numKeys,
-                        UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null,
-                    compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().size() == 0) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          // Create hardlink backups for the SST files that are going
+          // to be deleted after this RDB compaction.
+          for (String file : compactionJobInfo.inputFiles()) {
+            LOG.debug("Creating hard link for '{}'", file);
+            String saveLinkFileName =
+                sstBackupDir + new File(file).getName();
+            Path link = Paths.get(saveLinkFileName);
+            Path srcFile = Paths.get(file);
+            try {
+              Files.createLink(link, srcFile);
+            } catch (IOException e) {
+              LOG.error("Exception in creating hard link for {}", file);
+              throw new RuntimeException("Failed to create hard link", e);
             }
           }
-        };
 
-    list.add(onCompactionCompletedListener);
-    rocksOptions.setListeners(list);
+        }
+      }
+    };
   }
 
-  public RocksDB getRocksDBInstanceWithCompactionTracking(String dbPath)
-      throws RocksDBException {
-    final Options opt = new Options().setCreateIfMissing(true)
-        .setCompressionType(CompressionType.NO_COMPRESSION);
-    opt.setMaxBytesForLevelMultiplier(2);
-    setRocksDBForCompactionTracking(opt);
-    return RocksDB.open(opt, dbPath);
+  private AbstractEventListener newCompactionCompletedListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionCompleted(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().isEmpty()) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          final StringBuilder sb = new StringBuilder();
+
+          if (LOG.isDebugEnabled()) {
+            // Print compaction reason for this entry in the log file
+            // e.g. kLevelL0FilesNum / kLevelMaxLevelSize.
+            sb.append(COMPACTION_LOG_COMMENT_LINE_PREFIX)
+                .append(compactionJobInfo.compactionReason())
+                .append('\n');
+          }
+
+          // Mark the beginning of a compaction log
+          sb.append(COMPACTION_LOG_ENTRY_LINE_PREFIX);
+
+          // Trim DB path, only keep the SST file name
+          final int filenameOffset =
+              compactionJobInfo.inputFiles().get(0).lastIndexOf("/") + 1;
+
+          // Append the list of input files
+          final List<String> inputFiles = compactionJobInfo.inputFiles();
+          // Trim the file path, leave only the SST file name without extension
+          inputFiles.replaceAll(s -> s.substring(
+              filenameOffset, s.length() - SST_FILE_EXTENSION_LENGTH));
+          final String inputFilesJoined = String.join(",", inputFiles);
+          sb.append(inputFilesJoined);
+
+          // Insert delimiter between input files an output files
+          sb.append(':');
+
+          // Append the list of output files
+          final List<String> outputFiles = compactionJobInfo.outputFiles();
+          outputFiles.replaceAll(s -> s.substring(
+              filenameOffset, s.length() - SST_FILE_EXTENSION_LENGTH));
+          final String outputFilesJoined = String.join(",", outputFiles);
+          sb.append(outputFilesJoined);
+
+          // End of line
+          sb.append('\n');
+
+          // Write input and output file names to compaction log
+          appendToCurrentCompactionLog(sb.toString());
+
+          // Populate the DAG
+          populateCompactionDAG(inputFiles, outputFiles,
+              db.getLatestSequenceNumber());
+/*
+          if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
+            printMutableGraph(null, null, compactionDAGFwd);
+          }
+ */
+        }
+      }
+    };
   }
 
-  // Get a summary of a given SST file
-  public long getSSTFileSummary(String filename)
-      throws RocksDBException {
+  /**
+   * Get number of keys in an SST file.
+   * @param filename SST filename
+   * @return number of keys
+   */
+  private long getSSTFileSummary(String filename) throws RocksDBException {
+
+    if (!filename.endsWith(SST_FILE_EXTENSION)) {
+      filename += SST_FILE_EXTENSION;
+    }
+
     Options option = new Options();
     SstFileReader reader = new SstFileReader(option);
-    try {
-      reader.open(saveCompactedFilePath + filename);
-    } catch (RocksDBException e) {
-      reader.open(rocksDbPath + "/"+ filename);
+
+    File sstFile = new File(sstBackupDir + filename);
+    File sstFileInActiveDB = new File(activeDBLocationStr + filename);
+    if (sstFile.exists()) {
+      reader.open(sstBackupDir + filename);
+    } else if (sstFileInActiveDB.exists()) {
+      reader.open(activeDBLocationStr + filename);
+    } else {
+      throw new RuntimeException("Can't find SST file: " + filename);
     }
+
     TableProperties properties = reader.getTableProperties();
-    LOG.warn("getSSTFileSummary " + filename + ":: " +
-        properties.getNumEntries());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("{} has {} keys", filename, properties.getNumEntries());
+    }
     return properties.getNumEntries();
   }
 
-  // Read the current Live manifest for a given RocksDB instance (Active or
-  // Checkpoint). Returns the list of currently active SST FileNames.
-  @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
+  /**
+   * Helper method to trim the filename retrieved from LiveFileMetaData.
+   */
+  private String trimSSTFilename(String filename) {
+    if (!filename.startsWith("/")) {
+      final String errorMsg = String.format(
+          "Invalid start of filename: '%s'. Expected '/'", filename);
+      LOG.error(errorMsg);
+      throw new RuntimeException(errorMsg);
+    }
+    if (!filename.endsWith(SST_FILE_EXTENSION)) {
+      final String errorMsg = String.format(
+          "Invalid extension of file: '%s'. Expected '%s'",
+          filename, SST_FILE_EXTENSION_LENGTH);
+      LOG.error(errorMsg);
+      throw new RuntimeException(errorMsg);
+    }
+    return filename.substring("/".length(),
+        filename.length() - SST_FILE_EXTENSION_LENGTH);
+  }
+
+  /**
+   * Read the current Live manifest for a given RocksDB instance (Active or
+   * Checkpoint).
+   * @param dbPathArg path to a RocksDB directory
+   * @return a list of SST files (without extension) in the DB.
+   */
   public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     RocksDB rocksDB = null;
     HashSet<String> liveFiles = new HashSet<>();
-    //
-    try (final Options options =
-             new Options().setParanoidChecks(true)
-                 .setCreateIfMissing(true)
-                 .setCompressionType(CompressionType.NO_COMPRESSION)
-                 .setForceConsistencyChecks(false)) {
-      rocksDB = RocksDB.openReadOnly(options, dbPathArg);
+
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+    final List<ColumnFamilyDescriptor> cfd = new ArrayList<>();
+    cfd.add(new ColumnFamilyDescriptor(
+        "keyTable".getBytes(StandardCharsets.UTF_8)));

Review Comment:
   Good catch. I will add `directoryTable` and `fileTable` as well.
   
   I was following [PR #3885](https://github.com/apache/ozone/pull/3885/files#diff-babc9eea3d726784a8d1643c490a10c57d304232ad9c2a2cea30ebd807fe8c02R47-R49), which only tracks `keyTable` atm, meaning FSO keys and directories are ignored in that snapshot differ (for now? @nandakumar131 ).



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012542936


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -215,390 +288,511 @@ public static void addDebugLevel(Integer level) {
     DEBUG_LEVEL.add(level);
   }
 
-  // Flushes the WAL and Creates a RocksDB checkpoint
-  @SuppressWarnings({"NM_METHOD_NAMING_CONVENTION"})
-  public void createCheckPoint(String dbPathArg, String cpPathArg,
-                               RocksDB rocksDB) {
-    LOG.warn("Creating Checkpoint for RocksDB instance : " +
-        dbPathArg + "in a CheckPoint Location" + cpPathArg);
-    try {
-      rocksDB.flush(new FlushOptions());
-      Checkpoint cp = Checkpoint.create(rocksDB);
-      cp.createCheckpoint(cpPathArg);
-    } catch (RocksDBException e) {
-      throw new RuntimeException(e.getMessage());
+  /**
+   * Append (then flush) to the current compaction log file path.
+   * Note: This does NOT automatically append newline to the log.
+   */
+  private synchronized void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogPath == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Compaction log path is not set. "
+          + "Please check initialization.");
+      throw new RuntimeException("Compaction log path not set");
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogPath),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogPath, e);
     }
   }
 
-  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
-      throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
-            }
-          }
-        };
+  /**
+   * Append a sequence number to the compaction log (roughly) when an Ozone
+   * snapshot (RDB checkpoint) is taken.
+   * @param sequenceNum RDB sequence number
+   */
+  public void appendSequenceNumberToCompactionLog(long sequenceNum) {
+    final String line = COMPACTION_LOG_SEQNUM_LINE_PREFIX + sequenceNum + "\n";
+    appendToCurrentCompactionLog(line);
+  }
 
-    list.add(onCompactionCompletedListener);
+  /**
+   * Takes {@link org.rocksdb.Options}.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
     rocksOptions.setListeners(list);
   }
 
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
 
+  /**
+   * Takes {@link org.rocksdb.DBOptions}.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
 
-  public void setRocksDBForCompactionTracking(Options rocksOptions)
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      Options rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db,final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file);
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix,
-                      numKeys, currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix, numKeys,
-                        UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null,
-                    compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().size() == 0) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          // Create hardlink backups for the SST files that are going
+          // to be deleted after this RDB compaction.
+          for (String file : compactionJobInfo.inputFiles()) {
+            LOG.debug("Creating hard link for '{}'", file);
+            String saveLinkFileName =
+                sstBackupDir + new File(file).getName();
+            Path link = Paths.get(saveLinkFileName);
+            Path srcFile = Paths.get(file);
+            try {
+              Files.createLink(link, srcFile);
+            } catch (IOException e) {
+              LOG.error("Exception in creating hard link for {}", file);
+              throw new RuntimeException("Failed to create hard link", e);
             }
           }
-        };
 
-    list.add(onCompactionCompletedListener);
-    rocksOptions.setListeners(list);
+        }
+      }
+    };
   }
 
-  public RocksDB getRocksDBInstanceWithCompactionTracking(String dbPath)
-      throws RocksDBException {
-    final Options opt = new Options().setCreateIfMissing(true)
-        .setCompressionType(CompressionType.NO_COMPRESSION);
-    opt.setMaxBytesForLevelMultiplier(2);
-    setRocksDBForCompactionTracking(opt);
-    return RocksDB.open(opt, dbPath);
+  private AbstractEventListener newCompactionCompletedListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionCompleted(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().isEmpty()) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          final StringBuilder sb = new StringBuilder();
+
+          if (LOG.isDebugEnabled()) {
+            // Print compaction reason for this entry in the log file
+            // e.g. kLevelL0FilesNum / kLevelMaxLevelSize.
+            sb.append(COMPACTION_LOG_COMMENT_LINE_PREFIX)
+                .append(compactionJobInfo.compactionReason())
+                .append('\n');
+          }
+
+          // Mark the beginning of a compaction log
+          sb.append(COMPACTION_LOG_ENTRY_LINE_PREFIX);
+
+          // Trim DB path, only keep the SST file name

Review Comment:
   The full path is highly redundant. IMO there is no need to persist the full path as the SST file can only either be in the SST backup dir or in active DB dir. Storing the full path easily doubles the in-memory map memory consumption compared to file name only.
   
   Also when this listener is triggered, the input / output SST file path is always under the active DB. But those SST files may well have been deleted from active DB instance later, and those SSTs are only kept under the backup dir.
   
   The logic you described is already implemented in `getSSTFileSummary()`:
   
   https://github.com/apache/ozone/blob/bba897f83be886dc536b3fa81b0d437c17e00a14/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java#L467-L472



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012208796


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -664,21 +844,21 @@ public synchronized void printMutableGraph(
         topLevelNodes.add(n);
       }
     }
-    Iterator iter = topLevelNodes.iterator();
+    Iterator<CompactionNode> iter = topLevelNodes.iterator();
     while (iter.hasNext()) {

Review Comment:
   You are welcome to raise the refactoring comments. No worries at all. The code looks much cleaner with the suggestion. :D



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sadanand48 commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
sadanand48 commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012541012


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -215,390 +288,511 @@ public static void addDebugLevel(Integer level) {
     DEBUG_LEVEL.add(level);
   }
 
-  // Flushes the WAL and Creates a RocksDB checkpoint
-  @SuppressWarnings({"NM_METHOD_NAMING_CONVENTION"})
-  public void createCheckPoint(String dbPathArg, String cpPathArg,
-                               RocksDB rocksDB) {
-    LOG.warn("Creating Checkpoint for RocksDB instance : " +
-        dbPathArg + "in a CheckPoint Location" + cpPathArg);
-    try {
-      rocksDB.flush(new FlushOptions());
-      Checkpoint cp = Checkpoint.create(rocksDB);
-      cp.createCheckpoint(cpPathArg);
-    } catch (RocksDBException e) {
-      throw new RuntimeException(e.getMessage());
+  /**
+   * Append (then flush) to the current compaction log file path.
+   * Note: This does NOT automatically append newline to the log.
+   */
+  private synchronized void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogPath == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Compaction log path is not set. "
+          + "Please check initialization.");
+      throw new RuntimeException("Compaction log path not set");
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogPath),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogPath, e);
     }
   }
 
-  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
-      throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
-            }
-          }
-        };
+  /**
+   * Append a sequence number to the compaction log (roughly) when an Ozone
+   * snapshot (RDB checkpoint) is taken.
+   * @param sequenceNum RDB sequence number
+   */
+  public void appendSequenceNumberToCompactionLog(long sequenceNum) {
+    final String line = COMPACTION_LOG_SEQNUM_LINE_PREFIX + sequenceNum + "\n";
+    appendToCurrentCompactionLog(line);
+  }
 
-    list.add(onCompactionCompletedListener);
+  /**
+   * Takes {@link org.rocksdb.Options}.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
     rocksOptions.setListeners(list);
   }
 
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
 
+  /**
+   * Takes {@link org.rocksdb.DBOptions}.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
 
-  public void setRocksDBForCompactionTracking(Options rocksOptions)
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      Options rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db,final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file);
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix,
-                      numKeys, currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix, numKeys,
-                        UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null,
-                    compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().size() == 0) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          // Create hardlink backups for the SST files that are going
+          // to be deleted after this RDB compaction.
+          for (String file : compactionJobInfo.inputFiles()) {
+            LOG.debug("Creating hard link for '{}'", file);
+            String saveLinkFileName =
+                sstBackupDir + new File(file).getName();
+            Path link = Paths.get(saveLinkFileName);
+            Path srcFile = Paths.get(file);
+            try {
+              Files.createLink(link, srcFile);
+            } catch (IOException e) {
+              LOG.error("Exception in creating hard link for {}", file);
+              throw new RuntimeException("Failed to create hard link", e);
             }
           }
-        };
 
-    list.add(onCompactionCompletedListener);
-    rocksOptions.setListeners(list);
+        }
+      }
+    };
   }
 
-  public RocksDB getRocksDBInstanceWithCompactionTracking(String dbPath)
-      throws RocksDBException {
-    final Options opt = new Options().setCreateIfMissing(true)
-        .setCompressionType(CompressionType.NO_COMPRESSION);
-    opt.setMaxBytesForLevelMultiplier(2);
-    setRocksDBForCompactionTracking(opt);
-    return RocksDB.open(opt, dbPath);
+  private AbstractEventListener newCompactionCompletedListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionCompleted(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().isEmpty()) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          final StringBuilder sb = new StringBuilder();
+
+          if (LOG.isDebugEnabled()) {
+            // Print compaction reason for this entry in the log file
+            // e.g. kLevelL0FilesNum / kLevelMaxLevelSize.
+            sb.append(COMPACTION_LOG_COMMENT_LINE_PREFIX)
+                .append(compactionJobInfo.compactionReason())
+                .append('\n');
+          }
+
+          // Mark the beginning of a compaction log
+          sb.append(COMPACTION_LOG_ENTRY_LINE_PREFIX);
+
+          // Trim DB path, only keep the SST file name

Review Comment:
   Can we keep the absolute path here instead of just file name. This might be useful while opening the sst files in the later steps , if file is compacted and not present in the db, we can look it up from the hardlink dir.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sadanand48 commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
sadanand48 commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993079676


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,13 +529,151 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   * TODO: Drop synchronized? and make this thread safe?
+   */
+  private synchronized void processCompactionLogLine(String line) {
+    LOG.info("Processing line: {}", line);
+
+    // Skip comments
+    if (line.startsWith("#")) {
+      LOG.info("Skipped comment.");
+      return;
+    }
+
+    if (sstTokensRead == null) {
+      // Store the tokens in the first line
+      sstTokensRead = line.split("\t");
+      LOG.info("Length of inputFiles = {}", sstTokensRead.length);
+      if (sstTokensRead.length == 0) {
+        // Sanity check. inputFiles should never be empty. outputFiles could.
+        throw new RuntimeException(
+            "Compaction inputFiles list is empty. File is corrupted?");
+      }
+    } else {
+      final String[] outputFilesRead = line.split("\t");
+      LOG.info("Length of outputFiles = {}", outputFilesRead.length);
+
+      // Populate the compaction DAG
+      populateCompactionDAG(sstTokensRead, outputFilesRead);
+
+      // Reset inputFilesRead to null so
+      sstTokensRead = null;
+    }
+  }
+
+  private String[] sstTokensRead;
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   * <p>
+   * DO NOT use this function in another context without understanding what it
+   * does, due to the state preserved between calls (in sstTokensRead).
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> stream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      assert (sstTokensRead == null);
+      stream.forEach(this::processCompactionLogLine);
+      if (sstTokensRead != null) {
+        // Sanity check. Temp variable must be null after parsing.
+        // Otherwise it means the compaction log is corrupted.
+        throw new RuntimeException("Missing output files line. Corrupted?");
+      }
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Read compaction log until all dest (and src) db checkpoint SST
+   * nodes show up in the graph, or when it reaches the end of the log.
+   */
+  private boolean loadCompactionDAGBySSTSet(HashSet<String> sstSet) {
+
+    // Make a copy of the sstSet
+    HashSet<String> loadSet = new HashSet<>(sstSet);
+
+    // Check if all the nodes in the provided SST set is already loaded in DAG
+    for (String sstFile : sstSet) {
+      if (compactionNodeTable.containsKey(sstFile)) {
+        loadSet.remove(sstFile);
+      }
+    }
+
+    if (loadSet.size() == 0) {
+      // All expected nodes in the sstSet are already there,
+      //  no need to read any compaction log
+      return true;
+    }
+
+    // Otherwise, load compaction logs in order until all nodes are present in
+    //  the DAG.
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase().endsWith(".log"))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+
+          // TODO: Potential optimization: stop reading as soon as all nodes are
+          //  there. Currently it loads an entire file at a time.
+          readCompactionLogToDAG(logPath.toString());
+
+          for (Iterator<String> it = loadSet.iterator(); it.hasNext();) {
+            String sstFile = it.next();
+            if (compactionNodeTable.containsKey(sstFile)) {
+              LOG.debug("Found SST node: {}", sstFile);
+              it.remove();
+            }
+          }
+
+          if (loadSet.size() == 0) {
+            break;
+          }
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+
+    // Just in case there are still nodes to be expected not loaded.
+    if (loadSet.size() > 0) {
+      LOG.warn("The following nodes are missing from the compaction log: {}. "
+          + "Possibly because those a newly flushed SSTs that haven't gone "
+          + "though any compaction yet", loadSet);
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Given the src and destination Snapshots, it prints a Diff list.
+   *
+   * Expected input: src is a checkpoint taken AFTER dest checkpoint.
+   *
+   * @param src
+   * @param dest
+   * @throws RocksDBException
+   */
+  private synchronized List<String> printSnapdiffSSTFiles(
+      Snapshot src, Snapshot dest) {
+
+    LOG.warn("src db checkpoint: {}", src.dbPath);  // from
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
+    LOG.warn("dest db checkpoint: {}", dest.dbPath);  //to
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
+    System.out.println();
+
+    // Read compaction log until all dest (and src) db checkpoint SST
+    // nodes show up in the graph
+//    loadCompactionDAGBySSTSet(destSnapFiles);

Review Comment:
   Is this line commented out intentionally? we should load destSnapFiles also right?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993956506


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -129,6 +201,24 @@ public RocksDBCheckpointDiffer(String dbPath,
     cpPath = checkpointPath;
 
     saveCompactedFilePath = sstFileSaveDir;
+
+    // Append /
+    if (!saveCompactedFilePath.endsWith("/")) {
+      saveCompactedFilePath += "/";
+    }
+
+    // Create the directory if SST backup path does not already exist
+    File dir = new File(saveCompactedFilePath);
+    if (dir.exists()) {
+      deleteDirectory(dir);  // TODO: FOR EASE OF TESTING ONLY. DO NOT DELETE DIR WHEN MERGING
+    }
+    if (!dir.mkdir()) {
+      LOG.error("Failed to create SST file backup directory!");
+      // TODO: Throw unrecoverable exception and Crash OM ?
+      throw new RuntimeException("Failed to create SST file backup directory. "
+          + "Check write permission.");
+    }
+
     rocksDbPath = dbPath;
     cfDBPath = cfPath;

Review Comment:
   cfDBPath seems unused.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on PR #3824:
URL: https://github.com/apache/ozone/pull/3824#issuecomment-1276799800

   This is in response to the discussion here: https://github.com/apache/ozone/pull/3786#discussion_r992935645
   
   To me there is value in the conceptual simplicity of having all the state in rocksdb.
   
   And I don't think it would have to be a new RDB instance.  It could just be a separate CF in the existing one.  I know that that CF wouldn't be shared through ratis transactions, but it would be shared with followers when they get bootstrapped (which happens to be what we want.)
   
   Finally, with respect to the ordering problem mentioned above, couldn't the db lastSeqNum be used as the rocksdb key, (as it is used in the filename currently)?
   
   I don't feel too strongly about it, and will understand if you think it is not worth the effort.  But it does seem like a cleaner approach to me.
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1011111600


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -688,130 +868,139 @@ public synchronized void printMutableGraph(
       iter.remove();
       iter = topLevelNodes.iterator();
     }
-    LOG.warn("src snap:" + srcSnapId);
-    LOG.warn("dest snap:" + destSnapId);
+    LOG.debug("src snap: " + srcSnapId);
+    LOG.debug("dest snap: " + destSnapId);
     for (CompactionNode n : allNodes) {
-      LOG.warn("Files are :" + n.fileName);
+      LOG.debug("Files are: " + n.fileName);
     }
   }
 
+  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
+    return compactionDAGFwd;
+  }
 
-  public void createSnapshot(RocksDB rocksDB) throws InterruptedException {
-
-    LOG.warn("Current time is::" + System.currentTimeMillis());
-    long t1 = System.currentTimeMillis();
-
-    cpPath = cpPath + lastSnapshotCounter;
-    createCheckPoint(rocksDbPath, cpPath, rocksDB);
-    allSnapshots[lastSnapshotCounter] = new Snapshot(cpPath,
-    lastSnapshotPrefix, lastSnapshotCounter);
-
-    long t2 = System.currentTimeMillis();
-    LOG.warn("Current time is::" + t2);
-
-    LOG.warn("millisecond difference is ::" + (t2 - t1));
-   Thread.sleep(100);
-   ++lastSnapshotCounter;
-   lastSnapshotPrefix = "sid_" + lastSnapshotCounter;
-   LOG.warn("done :: 1");
+  public MutableGraph<CompactionNode> getCompactionReverseDAG() {
+    return compactionDAGReverse;
   }
 
+  /**
+   * Populate the compaction DAG with input and output SST files lists.
+   */
+  @SuppressFBWarnings({"AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION"})
+  private void populateCompactionDAG(List<String> inputFiles,
+      List<String> outputFiles, long seqNum) {
 
-  public void printAllSnapshots() throws InterruptedException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
-      }
-      LOG.warn("Snapshot id" + snap.snapshotID);
-      LOG.warn("Snapshot path" + snap.dbPath);
-      LOG.warn("Snapshot Generation" + snap.snapshotGeneration);
-      LOG.warn("");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Input {} -> Output {}", inputFiles, outputFiles);
     }
-  }
 
-  public void diffAllSnapshots() throws InterruptedException, RocksDBException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
+    for (String outfile : outputFiles) {
+      CompactionNode outfileNode = compactionNodeTable.get(outfile);
+      if (outfileNode == null) {
+        long numKeys = 0L;
+        try {
+          numKeys = getSSTFileSummary(outfile);
+        } catch (Exception e) {
+          LOG.warn("Exception in getSSTFileSummary: {}", e.getMessage());
+        }
+        outfileNode = new CompactionNode(outfile, null, numKeys, seqNum);
+        compactionDAGFwd.addNode(outfileNode);
+        compactionDAGReverse.addNode(outfileNode);
+        compactionNodeTable.put(outfile, outfileNode);
       }
-      printSnapdiffSSTFiles(allSnapshots[lastSnapshotCounter - 1], snap);
-    }
-  }
 
-  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
-    return compactionDAGFwd;
-  }
+      for (String infile : inputFiles) {
+        CompactionNode infileNode = compactionNodeTable.get(infile);
+        if (infileNode == null) {
+          long numKeys = 0L;
+          try {
+            numKeys = getSSTFileSummary(infile);
+          } catch (Exception e) {
+            LOG.warn("Exception in getSSTFileSummary: {}", e.getMessage());
+          }
+          infileNode = new CompactionNode(infile, null, numKeys, seqNum);
+          compactionDAGFwd.addNode(infileNode);
+          compactionDAGReverse.addNode(infileNode);
+          compactionNodeTable.put(infile, infileNode);
+        }
+        // Draw the edges
+        if (outfileNode.fileName.compareToIgnoreCase(
+            infileNode.fileName) != 0) {
+          compactionDAGFwd.putEdge(outfileNode, infileNode);
+          compactionDAGReverse.putEdge(infileNode, outfileNode);
+        }
+      }
+    }
 
-  public MutableGraph<CompactionNode> getCompactionReverseDAG() {
-    return compactionDAGFwd;
   }
 
+  @VisibleForTesting
   public synchronized void traverseGraph(
       MutableGraph<CompactionNode> reverseMutableGraph,
       MutableGraph<CompactionNode> fwdMutableGraph) {
 
-      List<CompactionNode> nodeList =
-        compactionNodeTable.values().stream().collect(Collectors.toList());
-    Collections.sort(nodeList, new NodeComparator());
+    List<CompactionNode> nodeList = compactionNodeTable.values().stream()
+        .sorted(new NodeComparator()).collect(Collectors.toList());
 
-    for (CompactionNode  infileNode : nodeList ) {
-      // fist go through fwdGraph to find nodes that don't have succesors.
+    for (CompactionNode infileNode : nodeList) {
+      // fist go through fwdGraph to find nodes that don't have successors.
       // These nodes will be the top level nodes in reverse graph
       Set<CompactionNode> successors = fwdMutableGraph.successors(infileNode);
-      if (successors == null || successors.size() == 0) {
-        LOG.warn("traverseGraph : No successors. cumulative " +
-            "keys : " + infileNode.cumulativeKeysReverseTraversal + "::total " +
-            "keys ::" + infileNode.totalNumberOfKeys);
+      if (successors.size() == 0) {
+        LOG.debug("No successors. Cumulative keys: {}, total keys: {}",
+            infileNode.cumulativeKeysReverseTraversal,
+            infileNode.totalNumberOfKeys);
         infileNode.cumulativeKeysReverseTraversal =
             infileNode.totalNumberOfKeys;
       }
     }
 
     HashSet<CompactionNode> visited = new HashSet<>();
-    for (CompactionNode  infileNode : nodeList ) {
+    for (CompactionNode infileNode : nodeList) {
       if (visited.contains(infileNode)) {
         continue;
       }
       visited.add(infileNode);
-      System.out.print("traverseGraph: Visiting node " + infileNode.fileName +
-          ":\n");
+      LOG.debug("Visiting node '{}'", infileNode.fileName);
       Set<CompactionNode> nextLevel = new HashSet<>();
       nextLevel.add(infileNode);
-      Set<CompactionNode> currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
+      Set<CompactionNode> currentLevel = new HashSet<>(nextLevel);
       nextLevel = new HashSet<>();
       int i = 1;
       while (currentLevel.size() != 0) {
-        LOG.warn("traverseGraph : DAG Level :" + i++);
+        LOG.debug("DAG Level {}", i++);
         for (CompactionNode current : currentLevel) {
-          LOG.warn("traverseGraph : expanding node " + current.fileName);
+          LOG.debug("Expanding node {}", current.fileName);
           Set<CompactionNode> successors =
               reverseMutableGraph.successors(current);
-          if (successors == null || successors.size() == 0) {
-            LOG.warn("traverseGraph : No successors. cumulative " +
-                "keys : " + current.cumulativeKeysReverseTraversal);
+          if (successors.size() == 0) {
+            LOG.debug("No successors. Cumulative keys: {}",
+                current.cumulativeKeysReverseTraversal);
           } else {
             for (CompactionNode oneSucc : successors) {
-              LOG.warn("traverseGraph : Adding to the next level : " +
-                  oneSucc.fileName);
-              LOG.warn("traverseGraph : " + oneSucc.fileName + "cum" + " keys"
-                  + oneSucc.cumulativeKeysReverseTraversal + "parent" + " " +
-                  current.fileName + " total " + current.totalNumberOfKeys);
+              LOG.debug("Adding to the next level: {}", oneSucc.fileName);
+              LOG.debug("'{}' cumulative keys: {}. parent '{}' total keys: {}",
+                  oneSucc.fileName, oneSucc.cumulativeKeysReverseTraversal,
+                  current.fileName, current.totalNumberOfKeys);
               oneSucc.cumulativeKeysReverseTraversal +=
                   current.cumulativeKeysReverseTraversal;
               nextLevel.add(oneSucc);
             }
           }
         }
-        currentLevel = new HashSet<>();
-        currentLevel.addAll(nextLevel);
+        currentLevel = new HashSet<>(nextLevel);
         nextLevel = new HashSet<>();
-        LOG.warn("");
       }
     }
   }
 
+  @VisibleForTesting
   public boolean debugEnabled(Integer level) {
     return DEBUG_LEVEL.contains(level);
   }
+
+  @VisibleForTesting
+  public static Logger getLog() {
+    return LOG;
+  }

Review Comment:
   This is exposed for UT to set log debug level. I could simply make `LOG` public or package-private but then realized this is the way in [other classes](https://github.com/apache/ozone/blob/649d7138177c5547a6d0c8cf4610a4e359ff0436/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java#L527-L530) as well. Either way it works.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1011111600


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -688,130 +868,139 @@ public synchronized void printMutableGraph(
       iter.remove();
       iter = topLevelNodes.iterator();
     }
-    LOG.warn("src snap:" + srcSnapId);
-    LOG.warn("dest snap:" + destSnapId);
+    LOG.debug("src snap: " + srcSnapId);
+    LOG.debug("dest snap: " + destSnapId);
     for (CompactionNode n : allNodes) {
-      LOG.warn("Files are :" + n.fileName);
+      LOG.debug("Files are: " + n.fileName);
     }
   }
 
+  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
+    return compactionDAGFwd;
+  }
 
-  public void createSnapshot(RocksDB rocksDB) throws InterruptedException {
-
-    LOG.warn("Current time is::" + System.currentTimeMillis());
-    long t1 = System.currentTimeMillis();
-
-    cpPath = cpPath + lastSnapshotCounter;
-    createCheckPoint(rocksDbPath, cpPath, rocksDB);
-    allSnapshots[lastSnapshotCounter] = new Snapshot(cpPath,
-    lastSnapshotPrefix, lastSnapshotCounter);
-
-    long t2 = System.currentTimeMillis();
-    LOG.warn("Current time is::" + t2);
-
-    LOG.warn("millisecond difference is ::" + (t2 - t1));
-   Thread.sleep(100);
-   ++lastSnapshotCounter;
-   lastSnapshotPrefix = "sid_" + lastSnapshotCounter;
-   LOG.warn("done :: 1");
+  public MutableGraph<CompactionNode> getCompactionReverseDAG() {
+    return compactionDAGReverse;
   }
 
+  /**
+   * Populate the compaction DAG with input and output SST files lists.
+   */
+  @SuppressFBWarnings({"AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION"})
+  private void populateCompactionDAG(List<String> inputFiles,
+      List<String> outputFiles, long seqNum) {
 
-  public void printAllSnapshots() throws InterruptedException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
-      }
-      LOG.warn("Snapshot id" + snap.snapshotID);
-      LOG.warn("Snapshot path" + snap.dbPath);
-      LOG.warn("Snapshot Generation" + snap.snapshotGeneration);
-      LOG.warn("");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Input {} -> Output {}", inputFiles, outputFiles);
     }
-  }
 
-  public void diffAllSnapshots() throws InterruptedException, RocksDBException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
+    for (String outfile : outputFiles) {
+      CompactionNode outfileNode = compactionNodeTable.get(outfile);
+      if (outfileNode == null) {
+        long numKeys = 0L;
+        try {
+          numKeys = getSSTFileSummary(outfile);
+        } catch (Exception e) {
+          LOG.warn("Exception in getSSTFileSummary: {}", e.getMessage());
+        }
+        outfileNode = new CompactionNode(outfile, null, numKeys, seqNum);
+        compactionDAGFwd.addNode(outfileNode);
+        compactionDAGReverse.addNode(outfileNode);
+        compactionNodeTable.put(outfile, outfileNode);
       }
-      printSnapdiffSSTFiles(allSnapshots[lastSnapshotCounter - 1], snap);
-    }
-  }
 
-  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
-    return compactionDAGFwd;
-  }
+      for (String infile : inputFiles) {
+        CompactionNode infileNode = compactionNodeTable.get(infile);
+        if (infileNode == null) {
+          long numKeys = 0L;
+          try {
+            numKeys = getSSTFileSummary(infile);
+          } catch (Exception e) {
+            LOG.warn("Exception in getSSTFileSummary: {}", e.getMessage());
+          }
+          infileNode = new CompactionNode(infile, null, numKeys, seqNum);
+          compactionDAGFwd.addNode(infileNode);
+          compactionDAGReverse.addNode(infileNode);
+          compactionNodeTable.put(infile, infileNode);
+        }
+        // Draw the edges
+        if (outfileNode.fileName.compareToIgnoreCase(
+            infileNode.fileName) != 0) {
+          compactionDAGFwd.putEdge(outfileNode, infileNode);
+          compactionDAGReverse.putEdge(infileNode, outfileNode);
+        }
+      }
+    }
 
-  public MutableGraph<CompactionNode> getCompactionReverseDAG() {
-    return compactionDAGFwd;
   }
 
+  @VisibleForTesting
   public synchronized void traverseGraph(
       MutableGraph<CompactionNode> reverseMutableGraph,
       MutableGraph<CompactionNode> fwdMutableGraph) {
 
-      List<CompactionNode> nodeList =
-        compactionNodeTable.values().stream().collect(Collectors.toList());
-    Collections.sort(nodeList, new NodeComparator());
+    List<CompactionNode> nodeList = compactionNodeTable.values().stream()
+        .sorted(new NodeComparator()).collect(Collectors.toList());
 
-    for (CompactionNode  infileNode : nodeList ) {
-      // fist go through fwdGraph to find nodes that don't have succesors.
+    for (CompactionNode infileNode : nodeList) {
+      // fist go through fwdGraph to find nodes that don't have successors.
       // These nodes will be the top level nodes in reverse graph
       Set<CompactionNode> successors = fwdMutableGraph.successors(infileNode);
-      if (successors == null || successors.size() == 0) {
-        LOG.warn("traverseGraph : No successors. cumulative " +
-            "keys : " + infileNode.cumulativeKeysReverseTraversal + "::total " +
-            "keys ::" + infileNode.totalNumberOfKeys);
+      if (successors.size() == 0) {
+        LOG.debug("No successors. Cumulative keys: {}, total keys: {}",
+            infileNode.cumulativeKeysReverseTraversal,
+            infileNode.totalNumberOfKeys);
         infileNode.cumulativeKeysReverseTraversal =
             infileNode.totalNumberOfKeys;
       }
     }
 
     HashSet<CompactionNode> visited = new HashSet<>();
-    for (CompactionNode  infileNode : nodeList ) {
+    for (CompactionNode infileNode : nodeList) {
       if (visited.contains(infileNode)) {
         continue;
       }
       visited.add(infileNode);
-      System.out.print("traverseGraph: Visiting node " + infileNode.fileName +
-          ":\n");
+      LOG.debug("Visiting node '{}'", infileNode.fileName);
       Set<CompactionNode> nextLevel = new HashSet<>();
       nextLevel.add(infileNode);
-      Set<CompactionNode> currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
+      Set<CompactionNode> currentLevel = new HashSet<>(nextLevel);
       nextLevel = new HashSet<>();
       int i = 1;
       while (currentLevel.size() != 0) {
-        LOG.warn("traverseGraph : DAG Level :" + i++);
+        LOG.debug("DAG Level {}", i++);
         for (CompactionNode current : currentLevel) {
-          LOG.warn("traverseGraph : expanding node " + current.fileName);
+          LOG.debug("Expanding node {}", current.fileName);
           Set<CompactionNode> successors =
               reverseMutableGraph.successors(current);
-          if (successors == null || successors.size() == 0) {
-            LOG.warn("traverseGraph : No successors. cumulative " +
-                "keys : " + current.cumulativeKeysReverseTraversal);
+          if (successors.size() == 0) {
+            LOG.debug("No successors. Cumulative keys: {}",
+                current.cumulativeKeysReverseTraversal);
           } else {
             for (CompactionNode oneSucc : successors) {
-              LOG.warn("traverseGraph : Adding to the next level : " +
-                  oneSucc.fileName);
-              LOG.warn("traverseGraph : " + oneSucc.fileName + "cum" + " keys"
-                  + oneSucc.cumulativeKeysReverseTraversal + "parent" + " " +
-                  current.fileName + " total " + current.totalNumberOfKeys);
+              LOG.debug("Adding to the next level: {}", oneSucc.fileName);
+              LOG.debug("'{}' cumulative keys: {}. parent '{}' total keys: {}",
+                  oneSucc.fileName, oneSucc.cumulativeKeysReverseTraversal,
+                  current.fileName, current.totalNumberOfKeys);
               oneSucc.cumulativeKeysReverseTraversal +=
                   current.cumulativeKeysReverseTraversal;
               nextLevel.add(oneSucc);
             }
           }
         }
-        currentLevel = new HashSet<>();
-        currentLevel.addAll(nextLevel);
+        currentLevel = new HashSet<>(nextLevel);
         nextLevel = new HashSet<>();
-        LOG.warn("");
       }
     }
   }
 
+  @VisibleForTesting
   public boolean debugEnabled(Integer level) {
     return DEBUG_LEVEL.contains(level);
   }
+
+  @VisibleForTesting
+  public static Logger getLog() {
+    return LOG;
+  }

Review Comment:
   This is for UT to set log debug level. I could simply make `LOG` public but then realized this is the way in other classes as well. Either way it works.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1011140236


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -664,21 +844,21 @@ public synchronized void printMutableGraph(
         topLevelNodes.add(n);
       }
     }
-    Iterator iter = topLevelNodes.iterator();
+    Iterator<CompactionNode> iter = topLevelNodes.iterator();
     while (iter.hasNext()) {

Review Comment:
   Thanks @hemantk-12 . I have incorporated the changes into the next commit.
   
   tbh I hadn't really touched this logic written by @prashantpogde. But it looks to me the method simply gathers all relevant nodes from src to dest snapshot (or ALL nodes if src is `null`). I have changed the debug message accordingly.
   
   CMIIW @prashantpogde 



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] sadanand48 commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
sadanand48 commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993317466


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,13 +529,151 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   * TODO: Drop synchronized? and make this thread safe?
+   */
+  private synchronized void processCompactionLogLine(String line) {
+    LOG.info("Processing line: {}", line);
+
+    // Skip comments
+    if (line.startsWith("#")) {
+      LOG.info("Skipped comment.");
+      return;
+    }
+
+    if (sstTokensRead == null) {
+      // Store the tokens in the first line
+      sstTokensRead = line.split("\t");
+      LOG.info("Length of inputFiles = {}", sstTokensRead.length);
+      if (sstTokensRead.length == 0) {
+        // Sanity check. inputFiles should never be empty. outputFiles could.
+        throw new RuntimeException(
+            "Compaction inputFiles list is empty. File is corrupted?");
+      }
+    } else {
+      final String[] outputFilesRead = line.split("\t");
+      LOG.info("Length of outputFiles = {}", outputFilesRead.length);
+
+      // Populate the compaction DAG
+      populateCompactionDAG(sstTokensRead, outputFilesRead);
+
+      // Reset inputFilesRead to null so
+      sstTokensRead = null;
+    }
+  }
+
+  private String[] sstTokensRead;
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   * <p>
+   * DO NOT use this function in another context without understanding what it
+   * does, due to the state preserved between calls (in sstTokensRead).
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> stream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      assert (sstTokensRead == null);
+      stream.forEach(this::processCompactionLogLine);
+      if (sstTokensRead != null) {
+        // Sanity check. Temp variable must be null after parsing.
+        // Otherwise it means the compaction log is corrupted.
+        throw new RuntimeException("Missing output files line. Corrupted?");
+      }
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Read compaction log until all dest (and src) db checkpoint SST
+   * nodes show up in the graph, or when it reaches the end of the log.
+   */
+  private boolean loadCompactionDAGBySSTSet(HashSet<String> sstSet) {
+
+    // Make a copy of the sstSet
+    HashSet<String> loadSet = new HashSet<>(sstSet);
+
+    // Check if all the nodes in the provided SST set is already loaded in DAG
+    for (String sstFile : sstSet) {
+      if (compactionNodeTable.containsKey(sstFile)) {
+        loadSet.remove(sstFile);
+      }
+    }
+
+    if (loadSet.size() == 0) {
+      // All expected nodes in the sstSet are already there,
+      //  no need to read any compaction log
+      return true;
+    }
+
+    // Otherwise, load compaction logs in order until all nodes are present in
+    //  the DAG.
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase().endsWith(".log"))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+
+          // TODO: Potential optimization: stop reading as soon as all nodes are
+          //  there. Currently it loads an entire file at a time.
+          readCompactionLogToDAG(logPath.toString());
+
+          for (Iterator<String> it = loadSet.iterator(); it.hasNext();) {
+            String sstFile = it.next();
+            if (compactionNodeTable.containsKey(sstFile)) {
+              LOG.debug("Found SST node: {}", sstFile);
+              it.remove();
+            }
+          }
+
+          if (loadSet.size() == 0) {
+            break;
+          }
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+
+    // Just in case there are still nodes to be expected not loaded.
+    if (loadSet.size() > 0) {
+      LOG.warn("The following nodes are missing from the compaction log: {}. "
+          + "Possibly because those a newly flushed SSTs that haven't gone "
+          + "though any compaction yet", loadSet);
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Given the src and destination Snapshots, it prints a Diff list.
+   *
+   * Expected input: src is a checkpoint taken AFTER dest checkpoint.
+   *
+   * @param src
+   * @param dest
+   * @throws RocksDBException
+   */
+  private synchronized List<String> printSnapdiffSSTFiles(
+      Snapshot src, Snapshot dest) {
+
+    LOG.warn("src db checkpoint: {}", src.dbPath);  // from
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
+    LOG.warn("dest db checkpoint: {}", dest.dbPath);  //to
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
+    System.out.println();
+
+    // Read compaction log until all dest (and src) db checkpoint SST
+    // nodes show up in the graph
+//    loadCompactionDAGBySSTSet(destSnapFiles);

Review Comment:
   Got it, we can maybe have a preconditions check to validate the generation for src and dest.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993951742


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -114,9 +113,82 @@ public class RocksDBCheckpointDiffer {
   private static final long UNKNOWN_COMPACTION_GEN = 0;
   private long currentCompactionGen = 0;
 
-  // Something to track all the snapshots created so far.
+  // Something to track all the snapshots created so far. TODO: REMOVE
   private Snapshot[] allSnapshots;
 
+  private String compactionLogParentDir = null;
+  private String compactionLogDir = null;
+
+  // Name of the directory to hold compaction logs (under parent dir)
+  private static final String COMPACTION_LOG_DIR = "compaction-log/";
+
+  // For DB compaction SST DAG persistence and reconstruction
+  // Should be initialized to the latest sequence number
+  private volatile String currentCompactionLogFilename = null;
+
+  private static final String COMPACTION_LOG_FILENAME_SUFFIX = ".log";
+
+  public void setCompactionLogParentDir(String parentDir) {
+    this.compactionLogParentDir = parentDir;
+
+    // Append /
+    if (!compactionLogParentDir.endsWith("/")) {
+      compactionLogParentDir += "/";
+    }
+
+    File pDir = new File(compactionLogParentDir);
+    if (!pDir.exists()) {
+      if (!pDir.mkdir()) {
+        LOG.error("Error creating compaction log parent dir.");
+        return;
+      }
+    }
+
+    compactionLogDir =
+        Paths.get(compactionLogParentDir, COMPACTION_LOG_DIR).toString();
+    File clDir = new File(compactionLogDir);
+    if (!clDir.mkdir()) {
+      LOG.error("Error creating compaction log dir.");
+      return;
+    }
+
+    // TODO: Write a README there explaining what the dir is for
+  }
+
+  private static final int LONG_MAX_STRLEN =
+      String.valueOf(Long.MAX_VALUE).length();
+
+  public void setCompactionLogFilenameBySeqNum(long latestSequenceId) {
+    String latestSequenceIdStr = String.valueOf(latestSequenceId);
+
+    if (latestSequenceIdStr.length() < LONG_MAX_STRLEN) {
+      // Pad zeroes to the left for sequential listing later
+      latestSequenceIdStr =
+          StringUtils.leftPad(latestSequenceIdStr, LONG_MAX_STRLEN, "0");
+    }
+
+    this.currentCompactionLogFilename = compactionLogParentDir +
+        COMPACTION_LOG_DIR + latestSequenceIdStr +
+        COMPACTION_LOG_FILENAME_SUFFIX;
+
+    // Create empty file when if it doesn't exist.
+    // Detect any write permission issue (if any) early.
+    File clFile = new File(currentCompactionLogFilename);
+    if (clFile.exists()) {
+      LOG.warn("Compaction log exists: {}. Will append",

Review Comment:
   It seems like things will break, if appendToCurrentCompactionLog() gets called in another thread, while this thread is right here between the if statement and the else statement.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993951742


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -114,9 +113,82 @@ public class RocksDBCheckpointDiffer {
   private static final long UNKNOWN_COMPACTION_GEN = 0;
   private long currentCompactionGen = 0;
 
-  // Something to track all the snapshots created so far.
+  // Something to track all the snapshots created so far. TODO: REMOVE
   private Snapshot[] allSnapshots;
 
+  private String compactionLogParentDir = null;
+  private String compactionLogDir = null;
+
+  // Name of the directory to hold compaction logs (under parent dir)
+  private static final String COMPACTION_LOG_DIR = "compaction-log/";
+
+  // For DB compaction SST DAG persistence and reconstruction
+  // Should be initialized to the latest sequence number
+  private volatile String currentCompactionLogFilename = null;
+
+  private static final String COMPACTION_LOG_FILENAME_SUFFIX = ".log";
+
+  public void setCompactionLogParentDir(String parentDir) {
+    this.compactionLogParentDir = parentDir;
+
+    // Append /
+    if (!compactionLogParentDir.endsWith("/")) {
+      compactionLogParentDir += "/";
+    }
+
+    File pDir = new File(compactionLogParentDir);
+    if (!pDir.exists()) {
+      if (!pDir.mkdir()) {
+        LOG.error("Error creating compaction log parent dir.");
+        return;
+      }
+    }
+
+    compactionLogDir =
+        Paths.get(compactionLogParentDir, COMPACTION_LOG_DIR).toString();
+    File clDir = new File(compactionLogDir);
+    if (!clDir.mkdir()) {
+      LOG.error("Error creating compaction log dir.");
+      return;
+    }
+
+    // TODO: Write a README there explaining what the dir is for
+  }
+
+  private static final int LONG_MAX_STRLEN =
+      String.valueOf(Long.MAX_VALUE).length();
+
+  public void setCompactionLogFilenameBySeqNum(long latestSequenceId) {
+    String latestSequenceIdStr = String.valueOf(latestSequenceId);
+
+    if (latestSequenceIdStr.length() < LONG_MAX_STRLEN) {
+      // Pad zeroes to the left for sequential listing later
+      latestSequenceIdStr =
+          StringUtils.leftPad(latestSequenceIdStr, LONG_MAX_STRLEN, "0");
+    }
+
+    this.currentCompactionLogFilename = compactionLogParentDir +
+        COMPACTION_LOG_DIR + latestSequenceIdStr +
+        COMPACTION_LOG_FILENAME_SUFFIX;
+
+    // Create empty file when if it doesn't exist.
+    // Detect any write permission issue (if any) early.
+    File clFile = new File(currentCompactionLogFilename);
+    if (clFile.exists()) {
+      LOG.warn("Compaction log exists: {}. Will append",

Review Comment:
   It seems like things will break, if appendToCurrentCompactionLog() gets called right here between the if statement and the else statement.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993955658


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -230,196 +330,154 @@ public void createCheckPoint(String dbPathArg, String cpPathArg,
     }
   }
 
+  /**
+   * Append (then flush) to the current compaction log file path.
+   */
+  public void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogFilename == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Current compaction log filename is not set. "
+          + "Please check initialization.");

Review Comment:
   To me, this error is serious enough to throw a runtime exception.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r993958198


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -746,19 +950,71 @@ public MutableGraph<CompactionNode> getCompactionReverseDAG() {
     return compactionDAGFwd;

Review Comment:
   compactionDAGReverse



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1010116545


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -230,196 +330,154 @@ public void createCheckPoint(String dbPathArg, String cpPathArg,
     }
   }
 
+  /**
+   * Append (then flush) to the current compaction log file path.
+   */
+  public void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogFilename == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Current compaction log filename is not set. "
+          + "Please check initialization.");
+      return;
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogFilename),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogFilename, e);
+    }
+  }
+
+  /**
+   * This takes Options.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  /**
+   * This takes DBOptions.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
+
   public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (db) {

Review Comment:
   Done



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1010116299


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -131,7 +132,30 @@ public static DBCheckpoint createOmSnapshotCheckpoint(
       OMMetadataManager omMetadataManager, SnapshotInfo snapshotInfo)
       throws IOException {
     RDBStore store = (RDBStore) omMetadataManager.getStore();
-    return store.getSnapshot(snapshotInfo.getCheckpointDirName());
+
+    final long dbLatestSequenceNumber =
+        ((RDBStore) omMetadataManager.getStore()).getDb()
+            .getLatestSequenceNumber();
+
+    final RocksDBCheckpointDiffer dbCpDiffer =
+        omMetadataManager.getStore().getRocksDBCheckpointDiffer();
+
+    final DBCheckpoint dbCheckpoint = store.getSnapshot(
+        snapshotInfo.getCheckpointDirName());
+
+    // Write snapshot generation (latest sequence number) to compaction log.
+    // This will be used for DAG reconstruction as snapshotGeneration.
+    dbCpDiffer.appendSequenceNumberToCompactionLog(dbLatestSequenceNumber);
+
+    // Set compaction log filename to the latest DB sequence number
+    // right after taking the RocksDB checkpoint for Ozone snapshot.
+    //
+    // Note it doesn't matter if sequence number hasn't increased (even though
+    // it shouldn't happen), since the writer always appends the file.
+    dbCpDiffer.setCompactionLogParentDir(store.getSnapshotsParentDir());

Review Comment:
   No it is not necessary. Thanks. Removed.



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012308690


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -215,253 +304,252 @@ public static void addDebugLevel(Integer level) {
     DEBUG_LEVEL.add(level);
   }
 
-  // Flushes the WAL and Creates a RocksDB checkpoint
-  @SuppressWarnings({"NM_METHOD_NAMING_CONVENTION"})
-  public void createCheckPoint(String dbPathArg, String cpPathArg,
-                               RocksDB rocksDB) {
-    LOG.warn("Creating Checkpoint for RocksDB instance : " +
-        dbPathArg + "in a CheckPoint Location" + cpPathArg);
-    try {
-      rocksDB.flush(new FlushOptions());
-      Checkpoint cp = Checkpoint.create(rocksDB);
-      cp.createCheckpoint(cpPathArg);
-    } catch (RocksDBException e) {
-      throw new RuntimeException(e.getMessage());
+  /**
+   * Append (then flush) to the current compaction log file path.
+   * Note: This does NOT automatically append newline to the log.
+   */
+  private synchronized void appendToCurrentCompactionLog(String content) {
+    if (currentCompactionLogPath == null) {
+      LOG.error("Unable to append compaction log. "
+          + "Compaction log path is not set. "
+          + "Please check initialization.");
+      throw new RuntimeException("Compaction log path not set");
+    }
+    try (BufferedWriter bw = Files.newBufferedWriter(
+        Paths.get(currentCompactionLogPath),
+        StandardOpenOption.CREATE, StandardOpenOption.APPEND)) {
+      bw.write(content);
+      bw.flush();
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to append compaction log to " +
+          currentCompactionLogPath, e);
     }
   }
 
-  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
-      throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      DBOptions rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db, final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file + ",");
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix, numKeys,
-                      currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix,
-                        numKeys, UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null, compactionDAGFwd);
-              }
-            }
-          }
-        };
+  /**
+   * Append a sequence number to the compaction log (roughly) when an Ozone
+   * snapshot (RDB checkpoint) is taken.
+   * @param sequenceNum RDB sequence number
+   */
+  public void appendSequenceNumberToCompactionLog(long sequenceNum) {
+    final String line = COMPACTION_LOG_SEQNUM_LINE_PREFIX + sequenceNum + "\n";
+    appendToCurrentCompactionLog(line);
+  }
 
-    list.add(onCompactionCompletedListener);
+  /**
+   * Takes {@link org.rocksdb.Options}.
+   */
+  public void setRocksDBForCompactionTracking(Options rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
     rocksOptions.setListeners(list);
   }
 
+  public void setRocksDBForCompactionTracking(Options rocksOptions) {
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
 
+  /**
+   * Takes {@link org.rocksdb.DBOptions}.
+   */
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions,
+      List<AbstractEventListener> list) {
+    list.add(newCompactionBeginListener());
+    list.add(newCompactionCompletedListener());
+    rocksOptions.setListeners(list);
+  }
 
-  public void setRocksDBForCompactionTracking(Options rocksOptions)
+  public void setRocksDBForCompactionTracking(DBOptions rocksOptions)
       throws RocksDBException {
-    setRocksDBForCompactionTracking(rocksOptions,
-        new ArrayList<AbstractEventListener>());
-  }
-
-  public void setRocksDBForCompactionTracking(
-      Options rocksOptions, List<AbstractEventListener> list) {
-    final AbstractEventListener onCompactionCompletedListener =
-        new AbstractEventListener() {
-          @Override
-          @SuppressFBWarnings({
-              "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
-              "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-          public void onCompactionCompleted(
-              final RocksDB db,final CompactionJobInfo compactionJobInfo) {
-            synchronized (db) {
-              LOG.warn(compactionJobInfo.compactionReason().toString());
-              LOG.warn("List of input files:");
-              for (String file : compactionJobInfo.inputFiles()) {
-                LOG.warn(file);
-                String saveLinkFileName =
-                    saveCompactedFilePath + new File(file).getName();
-                Path link = Paths.get(saveLinkFileName);
-                Path srcFile = Paths.get(file);
-                try {
-                  Files.createLink(link, srcFile);
-                } catch (IOException e) {
-                  LOG.warn("Exception in creating hardlink");
-                  e.printStackTrace();
-                }
-              }
-              LOG.warn("List of output files:");
-              for (String file : compactionJobInfo.outputFiles()) {
-                LOG.warn(file);
-              }
-              // Let us also build the graph
-              for (String outFilePath : compactionJobInfo.outputFiles()) {
-                String outfile =
-                    Paths.get(outFilePath).getFileName().toString();
-                CompactionNode outfileNode = compactionNodeTable.get(outfile);
-                if (outfileNode == null) {
-                  long numKeys = 0;
-                  try {
-                    numKeys = getSSTFileSummary(outfile);
-                  } catch (Exception e) {
-                    LOG.warn(e.getMessage());
-                  }
-                  outfileNode = new CompactionNode(outfile,
-                      lastSnapshotPrefix,
-                      numKeys, currentCompactionGen);
-                  compactionDAGFwd.addNode(outfileNode);
-                  compactionDAGReverse.addNode(outfileNode);
-                  compactionNodeTable.put(outfile, outfileNode);
-                }
-                for (String inFilePath : compactionJobInfo.inputFiles()) {
-                  String infile =
-                      Paths.get(inFilePath).getFileName().toString();
-                  CompactionNode infileNode = compactionNodeTable.get(infile);
-                  if (infileNode == null) {
-                    long numKeys = 0;
-                    try {
-                      numKeys = getSSTFileSummary(infile);
-                    } catch (Exception e) {
-                      LOG.warn(e.getMessage());
-                    }
-                    infileNode = new CompactionNode(infile,
-                        lastSnapshotPrefix, numKeys,
-                        UNKNOWN_COMPACTION_GEN);
-                    compactionDAGFwd.addNode(infileNode);
-                    compactionDAGReverse.addNode(infileNode);
-                    compactionNodeTable.put(infile, infileNode);
-                  }
-                  if (outfileNode.fileName.compareToIgnoreCase(
-                      infileNode.fileName) != 0) {
-                    compactionDAGFwd.putEdge(outfileNode, infileNode);
-                    compactionDAGReverse.putEdge(infileNode, outfileNode);
-                  }
-                }
-              }
-              if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
-                printMutableGraph(null, null,
-                    compactionDAGFwd);
-              }
+    setRocksDBForCompactionTracking(rocksOptions, new ArrayList<>());
+  }
+
+  private AbstractEventListener newCompactionBeginListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionBegin(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().size() == 0) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          // Create hardlink backups for the SST files that are going
+          // to be deleted after this RDB compaction.
+          for (String file : compactionJobInfo.inputFiles()) {
+            LOG.debug("Creating hard link for '{}'", file);
+            String saveLinkFileName =
+                sstBackupDir + new File(file).getName();
+            Path link = Paths.get(saveLinkFileName);
+            Path srcFile = Paths.get(file);
+            try {
+              Files.createLink(link, srcFile);
+            } catch (IOException e) {
+              LOG.error("Exception in creating hard link for {}", file);
+              throw new RuntimeException("Failed to create hard link", e);
             }
           }
-        };
 
-    list.add(onCompactionCompletedListener);
-    rocksOptions.setListeners(list);
+        }
+      }
+    };
   }
 
-  public RocksDB getRocksDBInstanceWithCompactionTracking(String dbPath)
-      throws RocksDBException {
-    final Options opt = new Options().setCreateIfMissing(true)
-        .setCompressionType(CompressionType.NO_COMPRESSION);
-    opt.setMaxBytesForLevelMultiplier(2);
-    setRocksDBForCompactionTracking(opt);
-    return RocksDB.open(opt, dbPath);
+  private AbstractEventListener newCompactionCompletedListener() {
+    return new AbstractEventListener() {
+      @Override
+      public void onCompactionCompleted(RocksDB db,
+          CompactionJobInfo compactionJobInfo) {
+
+        synchronized (compactionListenerWriteLock) {
+
+          if (compactionJobInfo.inputFiles().isEmpty()) {
+            LOG.error("Compaction input files list is empty");
+            return;
+          }
+
+          final StringBuilder sb = new StringBuilder();
+
+          if (LOG.isDebugEnabled()) {
+            // Print compaction reason for this entry in the log file
+            // e.g. kLevelL0FilesNum / kLevelMaxLevelSize.
+            sb.append(COMPACTION_LOG_COMMENT_LINE_PREFIX)
+                .append(compactionJobInfo.compactionReason())
+                .append('\n');
+          }
+
+          // Mark the beginning of a compaction log
+          sb.append(COMPACTION_LOG_ENTRY_LINE_PREFIX);
+
+          // Trim DB path, only keep the SST file name
+          final int filenameOffset =
+              compactionJobInfo.inputFiles().get(0).lastIndexOf("/") + 1;
+
+          // Append the list of input files
+          final List<String> inputFiles = compactionJobInfo.inputFiles();
+          // Trim the file path, leave only the SST file name without extension
+          inputFiles.replaceAll(s -> s.substring(
+              filenameOffset, s.length() - SST_FILE_EXTENSION_LENGTH));
+          final String inputFilesJoined = String.join(",", inputFiles);
+          sb.append(inputFilesJoined);
+
+          // Insert delimiter between input files an output files
+          sb.append(':');
+
+          // Append the list of output files
+          final List<String> outputFiles = compactionJobInfo.outputFiles();
+          outputFiles.replaceAll(s -> s.substring(
+              filenameOffset, s.length() - SST_FILE_EXTENSION_LENGTH));
+          final String outputFilesJoined = String.join(",", outputFiles);
+          sb.append(outputFilesJoined);
+
+          // End of line
+          sb.append('\n');
+
+          // Write input and output file names to compaction log
+          appendToCurrentCompactionLog(sb.toString());
+
+          // Populate the DAG
+          populateCompactionDAG(inputFiles, outputFiles,
+              db.getLatestSequenceNumber());
+/*
+          if (debugEnabled(DEBUG_DAG_BUILD_UP)) {
+            printMutableGraph(null, null, compactionDAGFwd);
+          }
+ */
+        }
+      }
+    };
   }
 
-  // Get a summary of a given SST file
-  public long getSSTFileSummary(String filename)
-      throws RocksDBException {
+  /**
+   * Get number of keys in an SST file.
+   * @param filename SST filename
+   * @return number of keys
+   */
+  private long getSSTFileSummary(String filename) throws RocksDBException {
+
+    if (!filename.endsWith(SST_FILE_EXTENSION)) {
+      filename += SST_FILE_EXTENSION;
+    }
+
     Options option = new Options();
     SstFileReader reader = new SstFileReader(option);
-    try {
-      reader.open(saveCompactedFilePath + filename);
-    } catch (RocksDBException e) {
-      reader.open(rocksDbPath + "/"+ filename);
+
+    File sstFile = new File(sstBackupDir + filename);
+    File sstFileInActiveDB = new File(activeDBLocationStr + filename);
+    if (sstFile.exists()) {
+      reader.open(sstBackupDir + filename);
+    } else if (sstFileInActiveDB.exists()) {
+      reader.open(activeDBLocationStr + filename);
+    } else {
+      throw new RuntimeException("Can't find SST file: " + filename);
     }
+
     TableProperties properties = reader.getTableProperties();
-    LOG.warn("getSSTFileSummary " + filename + ":: " +
-        properties.getNumEntries());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("{} has {} keys", filename, properties.getNumEntries());
+    }
     return properties.getNumEntries();
   }
 
-  // Read the current Live manifest for a given RocksDB instance (Active or
-  // Checkpoint). Returns the list of currently active SST FileNames.
-  @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
+  /**
+   * Helper method to trim the filename retrieved from LiveFileMetaData.
+   */
+  private String trimSSTFilename(String filename) {
+    if (!filename.startsWith("/")) {
+      final String errorMsg = String.format(
+          "Invalid start of filename: '%s'. Expected '/'", filename);
+      LOG.error(errorMsg);
+      throw new RuntimeException(errorMsg);
+    }
+    if (!filename.endsWith(SST_FILE_EXTENSION)) {
+      final String errorMsg = String.format(
+          "Invalid extension of file: '%s'. Expected '%s'",
+          filename, SST_FILE_EXTENSION_LENGTH);
+      LOG.error(errorMsg);
+      throw new RuntimeException(errorMsg);
+    }
+    return filename.substring("/".length(),
+        filename.length() - SST_FILE_EXTENSION_LENGTH);
+  }
+
+  /**
+   * Read the current Live manifest for a given RocksDB instance (Active or
+   * Checkpoint).
+   * @param dbPathArg path to a RocksDB directory
+   * @return a list of SST files (without extension) in the DB.
+   */
   public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     RocksDB rocksDB = null;
     HashSet<String> liveFiles = new HashSet<>();
-    //
-    try (final Options options =
-             new Options().setParanoidChecks(true)
-                 .setCreateIfMissing(true)
-                 .setCompressionType(CompressionType.NO_COMPRESSION)
-                 .setForceConsistencyChecks(false)) {
-      rocksDB = RocksDB.openReadOnly(options, dbPathArg);
+
+    final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
+    final List<ColumnFamilyDescriptor> cfd = new ArrayList<>();
+    cfd.add(new ColumnFamilyDescriptor(
+        "keyTable".getBytes(StandardCharsets.UTF_8)));

Review Comment:
   What about the file and directory tables?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on PR #3824:
URL: https://github.com/apache/ozone/pull/3824#issuecomment-1302374416

   I will merge this in a minute.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] smengcl commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1012349150


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.freon;
+
+import org.apache.hadoop.hdds.conf.DatanodeRatisServerConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig;
+import org.apache.hadoop.hdds.utils.db.RDBStore;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMStorage;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer;
+import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.DifferSnapshotInfo;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.server.RaftServer;
+import org.apache.ratis.server.raftlog.RaftLog;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
+import picocli.CommandLine;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_S3_VOLUME_NAME_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIR;
+
+/**
+ * Tests Freon, with MiniOzoneCluster.
+ */
+public class TestOMSnapshotDAG {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOMSnapshotDAG.class);
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+  private static ObjectStore store;
+  private final File metaDir = OMStorage.getOmDbDir(conf);
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting OZONE_ENABLED = true
+   *
+   */
+  @BeforeAll
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    DatanodeRatisServerConfig ratisServerConfig =
+        conf.getObject(DatanodeRatisServerConfig.class);
+    ratisServerConfig.setRequestTimeOut(Duration.ofSeconds(3));
+    ratisServerConfig.setWatchTimeOut(Duration.ofSeconds(3));
+    conf.setFromObject(ratisServerConfig);
+
+    RatisClientConfig.RaftConfig raftClientConfig =
+        conf.getObject(RatisClientConfig.RaftConfig.class);
+    raftClientConfig.setRpcRequestTimeout(Duration.ofSeconds(3));
+    raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(3));
+    conf.setFromObject(raftClientConfig);
+
+    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(5).build();
+    cluster.waitForClusterToBeReady();
+
+    store = cluster.getClient().getObjectStore();
+
+    GenericTestUtils.setLogLevel(RaftLog.LOG, Level.INFO);
+    GenericTestUtils.setLogLevel(RaftServer.LOG, Level.INFO);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterAll
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private String getDBCheckpointAbsolutePath(SnapshotInfo snapshotInfo) {
+    return metaDir + OM_KEY_PREFIX +
+        OM_SNAPSHOT_DIR + OM_KEY_PREFIX +
+        OM_DB_NAME + snapshotInfo.getCheckpointDirName();
+  }
+
+  private static String getSnapshotDBKey(String volumeName, String bucketName,
+      String snapshotName) {
+
+    final String dbKeyPrefix = OM_KEY_PREFIX + volumeName +
+        OM_KEY_PREFIX + bucketName;
+    return dbKeyPrefix + OM_KEY_PREFIX + snapshotName;
+  }
+
+  private DifferSnapshotInfo getDifferSnapshotInfo(
+      OMMetadataManager omMetadataManager, String volumeName, String bucketName,
+      String snapshotName) throws IOException {
+
+    final String dbKey = getSnapshotDBKey(volumeName, bucketName, snapshotName);
+    final SnapshotInfo snapshotInfo =
+        omMetadataManager.getSnapshotInfoTable().get(dbKey);
+    String checkpointPath = getDBCheckpointAbsolutePath(snapshotInfo);
+
+    // Use RocksDB transaction sequence number in SnapshotInfo, which is
+    // persisted at the time of snapshot creation, as the snapshot generation
+    return new DifferSnapshotInfo(checkpointPath, snapshotInfo.getSnapshotID(),
+        snapshotInfo.getDbTxSequenceNumber());
+  }
+
+  @Test
+  void testZeroSizeKey()
+      throws IOException, InterruptedException, TimeoutException {
+
+    RandomKeyGenerator randomKeyGenerator =
+        new RandomKeyGenerator(cluster.getConf());
+    CommandLine cmd = new CommandLine(randomKeyGenerator);
+    cmd.execute("--num-of-volumes", "1",
+        "--num-of-buckets", "1",
+        "--num-of-keys", "600",
+        "--num-of-threads", "1",
+        "--key-size", "0",
+        "--factor", "THREE",
+        "--type", "RATIS",
+        "--validate-writes"
+    );
+
+    Assertions.assertEquals(600L, randomKeyGenerator.getNumberOfKeysAdded());
+    Assertions.assertEquals(600L,
+        randomKeyGenerator.getSuccessfulValidationCount());
+
+    List<OmVolumeArgs> volList = cluster.getOzoneManager()
+        .listAllVolumes("", "", 10);
+    LOG.debug("List of all volumes: {}", volList);
+    final String volumeName = volList.stream().filter(e ->
+        !e.getVolume().equals(OZONE_S3_VOLUME_NAME_DEFAULT))  // Ignore s3v vol
+        .collect(Collectors.toList()).get(0).getVolume();
+    List<OmBucketInfo> bucketList =
+        cluster.getOzoneManager().listBuckets(volumeName, "", "", 10);
+    LOG.debug("List of all buckets under the first volume: {}", bucketList);
+    final String bucketName = bucketList.get(0).getBucketName();
+
+    // Create snapshot
+    String resp = store.createSnapshot(volumeName, bucketName, "snap1");
+    LOG.debug("Snapshot created: {}", resp);
+
+    final OzoneVolume volume = store.getVolume(volumeName);
+    final OzoneBucket bucket = volume.getBucket(bucketName);
+
+    for (int i = 0; i < 6000; i++) {
+      bucket.createKey("b_" + i, 0).close();
+    }
+
+    // Create another snapshot
+    resp = store.createSnapshot(volumeName, bucketName, "snap3");
+    LOG.debug("Snapshot created: {}", resp);
+
+    // Get snapshot SST diff list
+    OzoneManager ozoneManager = cluster.getOzoneManager();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    RDBStore rdbStore = (RDBStore) omMetadataManager.getStore();
+    RocksDBCheckpointDiffer differ = rdbStore.getRocksDBCheckpointDiffer();
+
+    DifferSnapshotInfo snap1 = getDifferSnapshotInfo(omMetadataManager,
+        volumeName, bucketName, "snap1");
+    DifferSnapshotInfo snap3 = getDifferSnapshotInfo(omMetadataManager,
+        volumeName, bucketName, "snap3");
+
+    // RocksDB does checkpointing in a separate thread, wait for it
+    final File checkpointSnap1 = new File(snap1.getDbPath());
+    GenericTestUtils.waitFor(checkpointSnap1::exists, 2000, 20000);
+    final File checkpointSnap3 = new File(snap3.getDbPath());
+    GenericTestUtils.waitFor(checkpointSnap3::exists, 2000, 20000);
+
+    List<String> actualDiffList = differ.getSSTDiffList(snap3, snap1);
+    LOG.debug("Got diff list: {}", actualDiffList);
+    final List<String> expectedDiffList = Arrays.asList("000059");

Review Comment:
   Added comment. Thanks!



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] hemantk-12 commented on a diff in pull request #3824: HDDS-7281. [Snapshot] Handle RocksDB compaction DAG persistence and reconstruction

Posted by GitBox <gi...@apache.org>.
hemantk-12 commented on code in PR #3824:
URL: https://github.com/apache/ozone/pull/3824#discussion_r1011049594


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -608,52 +792,48 @@ public Comparator<CompactionNode> reversed() {
     }
   }
 
-
-  public void dumpCompactioNodeTable() {
-    List<CompactionNode> nodeList =
-        compactionNodeTable.values().stream().collect(Collectors.toList());
-    Collections.sort(nodeList, new NodeComparator());
-    for (CompactionNode n : nodeList ) {
-      LOG.warn("File : " + n.fileName + " :: Total keys : "
-          + n.totalNumberOfKeys);
-      LOG.warn("File : " + n.fileName + " :: Cumulative keys : "  +
+  @VisibleForTesting
+  public void dumpCompactionNodeTable() {
+    List<CompactionNode> nodeList = compactionNodeTable.values().stream()
+        .sorted(new NodeComparator()).collect(Collectors.toList());
+    for (CompactionNode n : nodeList) {
+      LOG.info("File '{}' total keys: {}", n.fileName, n.totalNumberOfKeys);
+      LOG.info("File '{}' cumulative keys: {}", n.fileName,
           n.cumulativeKeysReverseTraversal);
     }
   }
 
+  @VisibleForTesting
   @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-  public synchronized void printMutableGraphFromAGivenNode(
-      String fileName, int level, MutableGraph<CompactionNode> mutableGraph) {
+  public synchronized void printMutableGraphFromAGivenNode(String fileName,
+      int level, MutableGraph<CompactionNode> mutableGraph) {
     CompactionNode infileNode =
         compactionNodeTable.get(Paths.get(fileName).getFileName().toString());
     if (infileNode == null) {
       return;
     }
-    System.out.print("\nCompaction Level : " + level + " Expandin File:" +
-        fileName + ":\n");
+    LOG.info("\nCompaction Level: " + level + " Expanding File: " + fileName);
     Set<CompactionNode> nextLevel = new HashSet<>();
     nextLevel.add(infileNode);
-    Set<CompactionNode> currentLevel = new HashSet<>();
-    currentLevel.addAll(nextLevel);
+    Set<CompactionNode> currentLevel = new HashSet<>(nextLevel);
     int i = 1;
     while (currentLevel.size() != 0) {
-      LOG.warn("DAG Level :" + i++);
+      LOG.info("DAG Level: " + i++);
+      StringBuilder sb = new StringBuilder();
       for (CompactionNode current : currentLevel) {
         Set<CompactionNode> successors = mutableGraph.successors(current);
         for (CompactionNode oneSucc : successors) {
-          System.out.print(oneSucc.fileName + " ");
+          sb.append(oneSucc.fileName).append(" ");
           nextLevel.add(oneSucc);
         }
       }
-      currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
+      LOG.info(sb.toString());
+      currentLevel = new HashSet<>(nextLevel);
       nextLevel = new HashSet<>();
-      LOG.warn("");
     }
   }
 
-  public synchronized void printMutableGraph(
-      String srcSnapId, String destSnapId,
+  synchronized void printMutableGraph(String srcSnapId, String destSnapId,
       MutableGraph<CompactionNode> mutableGraph) {
     LOG.warn("Printing the Graph");

Review Comment:
   Is it supposed to be `WARN`?



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,134 +558,231 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   */
+  private synchronized void processCompactionLogLine(String line) {
+
+    LOG.debug("Processing line: {}", line);
+
+    if (line.startsWith("#")) {
+      // Skip comments
+      LOG.debug("Comment line, skipped");
+    } else if (line.startsWith(COMPACTION_LOG_SEQNUM_LINE_PREFIX)) {
+      // Read sequence number
+      LOG.debug("Reading sequence number as snapshot generation");
+      final String seqNumStr =
+          line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length()).trim();
+      // This would the snapshot generation for the nodes to come
+      reconstructionSnapshotGeneration = Long.parseLong(seqNumStr);
+    } else if (line.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) {
+      // Read compaction log entry
+
+      // Trim the beginning
+      line = line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length());
+      final String[] io = line.split(":");
+      if (io.length != 2) {
+        LOG.error("Invalid line in compaction log: {}", line);
+        return;
+      }
+      final String[] inputFiles = io[0].split(",");
+      final String[] outputFiles = io[1].split(",");
+      populateCompactionDAG(Arrays.asList(inputFiles),
+          Arrays.asList(outputFiles), reconstructionSnapshotGeneration);
+    } else {
+      LOG.error("Invalid line in compaction log: {}", line);
+    }
+  }
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> logLineStream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      logLineStream.forEach(this::processCompactionLogLine);
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Load existing compaction log files to the in-memory DAG.
+   * This only needs to be done once during OM startup.
+   */
+  public synchronized void loadAllCompactionLogs() {
+    if (compactionLogDir == null) {
+      throw new RuntimeException("Compaction log directory must be set first");
+    }
+    reconstructionSnapshotGeneration = 0L;
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase()
+              .endsWith(COMPACTION_LOG_FILENAME_SUFFIX))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+          readCompactionLogToDAG(logPath.toString());
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+  }
+
+  /**
+   * Snapshot information node class in the DAG.
+   */
+  static class Snapshot {
+    private final String dbPath;
+    private final String snapshotID;
+    private final long snapshotGeneration;
+
+    Snapshot(String db, String id, long gen) {
+      dbPath = db;
+      snapshotID = id;
+      snapshotGeneration = gen;
+    }
+
+    public String getDbPath() {
+      return dbPath;
+    }
+
+    public String getSnapshotID() {
+      return snapshotID;
+    }
+
+    public long getSnapshotGeneration() {
+      return snapshotGeneration;
+    }
+  }
+
+  /**
+   * Get a list of SST files that differs between src and destination snapshots.
+   * <p>
+   * Expected input: src is a snapshot taken AFTER the dest.
+   *
+   * @param src source snapshot
+   * @param dest destination snapshot
+   */
+  public synchronized List<String> getSSTDiffList(Snapshot src, Snapshot dest) {
+
+    LOG.debug("src '{}' -> dest '{}'", src.dbPath, dest.dbPath);
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
 
     HashSet<String> fwdDAGSameFiles = new HashSet<>();
     HashSet<String> fwdDAGDifferentFiles = new HashSet<>();
 
-    LOG.warn("Doing forward diff between source and destination " +
-        "Snapshots:" + src.dbPath + ", " + dest.dbPath);
-    realPrintSnapdiffSSTFiles(src, dest, srcSnapFiles, destSnapFiles,
-        compactionDAGFwd,
-        fwdDAGSameFiles,
-        fwdDAGDifferentFiles);
+    LOG.debug("Doing forward diff between src and dest snapshots: " +
+        src.dbPath + " to " + dest.dbPath);
+    internalGetSSTDiffList(src, dest, srcSnapFiles, destSnapFiles,
+        compactionDAGFwd, fwdDAGSameFiles, fwdDAGDifferentFiles);
 
-    LOG.warn("Overall Summary \n" +
-            "Doing Overall diff between source and destination Snapshots:" +
-        src.dbPath + ", " + dest.dbPath);
-    System.out.print("fwd DAG Same files :");
-    for (String file : fwdDAGSameFiles) {
-      System.out.print(file + ", ");
-    }
-    LOG.warn("");
-    System.out.print("\nFwd DAG Different files :");
-    for (String file : fwdDAGDifferentFiles) {
-      CompactionNode n = compactionNodeTable.get(file);
-      System.out.print(file + ", ");
+    List<String> res = new ArrayList<>();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Result of diff from src '" + src.dbPath + "' to dest '" +
+          dest.dbPath + "':");
+      StringBuilder logSB = new StringBuilder();
+
+      logSB.append("Fwd DAG same SST files:      ");
+      for (String file : fwdDAGSameFiles) {
+        logSB.append(file).append(" ");
+      }
+      LOG.debug(logSB.toString());
+
+      logSB.setLength(0);
+      logSB.append("Fwd DAG different SST files: ");
+      for (String file : fwdDAGDifferentFiles) {
+        logSB.append(file).append(" ");
+        res.add(file);
+      }
+      LOG.debug(logSB.toString());
+
+    } else {
+      res.addAll(fwdDAGDifferentFiles);
     }
-    LOG.warn("");
+
+    return res;
   }
 
+  /**
+   * Core getSSTDiffList logic.
+   */
   @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-  public synchronized void realPrintSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest,
-      HashSet<String> srcSnapFiles,
-      HashSet<String> destSnapFiles,
+  private void internalGetSSTDiffList(Snapshot src, Snapshot dest,
+      HashSet<String> srcSnapFiles, HashSet<String> destSnapFiles,
       MutableGraph<CompactionNode> mutableGraph,
       HashSet<String> sameFiles, HashSet<String> differentFiles) {
 
-
     for (String fileName : srcSnapFiles) {
       if (destSnapFiles.contains(fileName)) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + " and Dest " +
-            "Snapshot" + dest.dbPath + " Contain Same file " + fileName);
+        LOG.debug("Source '{}' and destination '{}' share the same SST '{}'",
+            src.dbPath, dest.dbPath, fileName);
         sameFiles.add(fileName);
         continue;
       }
       CompactionNode infileNode =
           compactionNodeTable.get(Paths.get(fileName).getFileName().toString());
       if (infileNode == null) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + "File " + fileName + "was " +
-            "never compacted");
+        LOG.debug("Src " + src.dbPath + " File " + fileName +
+            " was never compacted");
         differentFiles.add(fileName);
         continue;
       }
-      System.out.print(" Expandin File:" + fileName + ":\n");
-      Set<CompactionNode> nextLevel = new HashSet<>();
-      nextLevel.add(infileNode);
+      LOG.debug("Expanding SST file: " + fileName);
       Set<CompactionNode> currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
-      nextLevel = new HashSet<>();
+      currentLevel.add(infileNode);
+      Set<CompactionNode> nextLevel = new HashSet<>();
       int i = 1;
       while (currentLevel.size() != 0) {
-        LOG.warn("DAG Level :" + i++);
+        LOG.debug("DAG Level: " + i++);
         for (CompactionNode current : currentLevel) {
-          LOG.warn("acknowledging file " + current.fileName);
+          LOG.debug("Acknowledging file " + current.fileName);
           if (current.snapshotGeneration <= dest.snapshotGeneration) {
-            LOG.warn("Reached dest generation count. SrcSnapshot : " +
-                src.dbPath + " and Dest " + "Snapshot" + dest.dbPath +
-                " Contain Diffrent file " + current.fileName);
+            LOG.debug("Reached dest generation count. Src: " +
+                src.dbPath + " and Dest: " + dest.dbPath +
+                " have different file: " + current.fileName);
             differentFiles.add(current.fileName);
             continue;
           }
           Set<CompactionNode> successors = mutableGraph.successors(current);
-          if (successors == null || successors.size() == 0) {
-            LOG.warn("No further compaction for the file" +
-                ".SrcSnapshot : " + src.dbPath + " and Dest " +
-                "Snapshot" + dest.dbPath + " Contain Diffrent file " +
-                current.fileName);
+          if (successors.size() == 0) {

Review Comment:
   1. I would prefer to use`isEmpty`. Is `successors` nullable? If yes, you can use `CollectionUtils.isEmpty()`.
   2. you can flatten it by using continue;
   ```
   if (successors.isEmpty()) {
     ...
     continue;
   }
   // else not needed.
   ```



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -664,21 +844,21 @@ public synchronized void printMutableGraph(
         topLevelNodes.add(n);
       }
     }
-    Iterator iter = topLevelNodes.iterator();
+    Iterator<CompactionNode> iter = topLevelNodes.iterator();
     while (iter.hasNext()) {

Review Comment:
   It would be much easier to understand if we use Queue instead of set/iterator and resetting it at multiple places.
   
   ```
     synchronized void printMutableGraph(String srcSnapId, String destSnapId,
                                         MutableGraph<CompactionNode> mutableGraph) {
       LOG.warn("Printing the Graph");
       Queue<CompactionNode> topLevelNodes = new LinkedList<>();
       Set<CompactionNode> allNodes = new HashSet<>();
       for (CompactionNode n : mutableGraph.nodes()) {
         if (srcSnapId == null ||
                 n.snapshotId.compareToIgnoreCase(srcSnapId) == 0) {
           topLevelNodes.add(n);
         }
       }
   
       while (!topLevelNodes .isEmpty()) {
         CompactionNode n = topLevelNodes.poll();
         Set<CompactionNode> succ = mutableGraph.successors(n);
         LOG.debug("Parent Node: " + n.fileName);
         if (succ.size() == 0) {
           LOG.debug("No child node");
           allNodes.add(n);
           continue;
         }
         for (CompactionNode oneSucc : succ) {
           LOG.debug("Children Node: " + oneSucc.fileName);
           if (srcSnapId == null ||
                   oneSucc.snapshotId.compareToIgnoreCase(destSnapId) == 0) {
             allNodes.add(oneSucc);
             continue;
           }
           topLevelNodes.add(oneSucc);
         }
       }
   
       LOG.debug("src snap: " + srcSnapId);
       LOG.debug("dest snap: " + destSnapId);
       for (CompactionNode n : allNodes) {
         LOG.debug("Files are: " + n.fileName);
       }
     }
   ```



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,134 +558,231 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   */
+  private synchronized void processCompactionLogLine(String line) {
+
+    LOG.debug("Processing line: {}", line);
+
+    if (line.startsWith("#")) {
+      // Skip comments
+      LOG.debug("Comment line, skipped");
+    } else if (line.startsWith(COMPACTION_LOG_SEQNUM_LINE_PREFIX)) {
+      // Read sequence number
+      LOG.debug("Reading sequence number as snapshot generation");
+      final String seqNumStr =
+          line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length()).trim();
+      // This would the snapshot generation for the nodes to come
+      reconstructionSnapshotGeneration = Long.parseLong(seqNumStr);
+    } else if (line.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) {
+      // Read compaction log entry
+
+      // Trim the beginning
+      line = line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length());
+      final String[] io = line.split(":");
+      if (io.length != 2) {
+        LOG.error("Invalid line in compaction log: {}", line);
+        return;
+      }
+      final String[] inputFiles = io[0].split(",");
+      final String[] outputFiles = io[1].split(",");
+      populateCompactionDAG(Arrays.asList(inputFiles),
+          Arrays.asList(outputFiles), reconstructionSnapshotGeneration);
+    } else {
+      LOG.error("Invalid line in compaction log: {}", line);
+    }
+  }
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> logLineStream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      logLineStream.forEach(this::processCompactionLogLine);
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Load existing compaction log files to the in-memory DAG.
+   * This only needs to be done once during OM startup.
+   */
+  public synchronized void loadAllCompactionLogs() {
+    if (compactionLogDir == null) {
+      throw new RuntimeException("Compaction log directory must be set first");
+    }
+    reconstructionSnapshotGeneration = 0L;
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase()
+              .endsWith(COMPACTION_LOG_FILENAME_SUFFIX))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+          readCompactionLogToDAG(logPath.toString());
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+  }
+
+  /**
+   * Snapshot information node class in the DAG.
+   */
+  static class Snapshot {
+    private final String dbPath;
+    private final String snapshotID;
+    private final long snapshotGeneration;
+
+    Snapshot(String db, String id, long gen) {
+      dbPath = db;
+      snapshotID = id;
+      snapshotGeneration = gen;
+    }
+
+    public String getDbPath() {
+      return dbPath;
+    }
+
+    public String getSnapshotID() {
+      return snapshotID;
+    }
+
+    public long getSnapshotGeneration() {
+      return snapshotGeneration;
+    }
+  }
+
+  /**
+   * Get a list of SST files that differs between src and destination snapshots.
+   * <p>
+   * Expected input: src is a snapshot taken AFTER the dest.
+   *
+   * @param src source snapshot
+   * @param dest destination snapshot
+   */
+  public synchronized List<String> getSSTDiffList(Snapshot src, Snapshot dest) {
+
+    LOG.debug("src '{}' -> dest '{}'", src.dbPath, dest.dbPath);
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
 
     HashSet<String> fwdDAGSameFiles = new HashSet<>();
     HashSet<String> fwdDAGDifferentFiles = new HashSet<>();
 
-    LOG.warn("Doing forward diff between source and destination " +
-        "Snapshots:" + src.dbPath + ", " + dest.dbPath);
-    realPrintSnapdiffSSTFiles(src, dest, srcSnapFiles, destSnapFiles,
-        compactionDAGFwd,
-        fwdDAGSameFiles,
-        fwdDAGDifferentFiles);
+    LOG.debug("Doing forward diff between src and dest snapshots: " +
+        src.dbPath + " to " + dest.dbPath);
+    internalGetSSTDiffList(src, dest, srcSnapFiles, destSnapFiles,
+        compactionDAGFwd, fwdDAGSameFiles, fwdDAGDifferentFiles);
 
-    LOG.warn("Overall Summary \n" +
-            "Doing Overall diff between source and destination Snapshots:" +
-        src.dbPath + ", " + dest.dbPath);
-    System.out.print("fwd DAG Same files :");
-    for (String file : fwdDAGSameFiles) {
-      System.out.print(file + ", ");
-    }
-    LOG.warn("");
-    System.out.print("\nFwd DAG Different files :");
-    for (String file : fwdDAGDifferentFiles) {
-      CompactionNode n = compactionNodeTable.get(file);
-      System.out.print(file + ", ");
+    List<String> res = new ArrayList<>();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Result of diff from src '" + src.dbPath + "' to dest '" +
+          dest.dbPath + "':");
+      StringBuilder logSB = new StringBuilder();
+
+      logSB.append("Fwd DAG same SST files:      ");
+      for (String file : fwdDAGSameFiles) {
+        logSB.append(file).append(" ");
+      }
+      LOG.debug(logSB.toString());
+
+      logSB.setLength(0);
+      logSB.append("Fwd DAG different SST files: ");
+      for (String file : fwdDAGDifferentFiles) {
+        logSB.append(file).append(" ");
+        res.add(file);
+      }
+      LOG.debug(logSB.toString());
+
+    } else {
+      res.addAll(fwdDAGDifferentFiles);
     }
-    LOG.warn("");
+
+    return res;
   }
 
+  /**
+   * Core getSSTDiffList logic.
+   */
   @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-  public synchronized void realPrintSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest,
-      HashSet<String> srcSnapFiles,
-      HashSet<String> destSnapFiles,
+  private void internalGetSSTDiffList(Snapshot src, Snapshot dest,
+      HashSet<String> srcSnapFiles, HashSet<String> destSnapFiles,
       MutableGraph<CompactionNode> mutableGraph,
       HashSet<String> sameFiles, HashSet<String> differentFiles) {
 
-
     for (String fileName : srcSnapFiles) {
       if (destSnapFiles.contains(fileName)) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + " and Dest " +
-            "Snapshot" + dest.dbPath + " Contain Same file " + fileName);
+        LOG.debug("Source '{}' and destination '{}' share the same SST '{}'",
+            src.dbPath, dest.dbPath, fileName);
         sameFiles.add(fileName);
         continue;
       }
       CompactionNode infileNode =
           compactionNodeTable.get(Paths.get(fileName).getFileName().toString());
       if (infileNode == null) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + "File " + fileName + "was " +
-            "never compacted");
+        LOG.debug("Src " + src.dbPath + " File " + fileName +
+            " was never compacted");
         differentFiles.add(fileName);
         continue;
       }
-      System.out.print(" Expandin File:" + fileName + ":\n");
-      Set<CompactionNode> nextLevel = new HashSet<>();
-      nextLevel.add(infileNode);
+      LOG.debug("Expanding SST file: " + fileName);
       Set<CompactionNode> currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
-      nextLevel = new HashSet<>();
+      currentLevel.add(infileNode);
+      Set<CompactionNode> nextLevel = new HashSet<>();
       int i = 1;
       while (currentLevel.size() != 0) {

Review Comment:
   nit: `isEmpty` is better from readability perspective. 
   ```
   while (!currentLevel.isEmpty()) {
     ...
   }
   ```



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -688,130 +868,139 @@ public synchronized void printMutableGraph(
       iter.remove();
       iter = topLevelNodes.iterator();
     }
-    LOG.warn("src snap:" + srcSnapId);
-    LOG.warn("dest snap:" + destSnapId);
+    LOG.debug("src snap: " + srcSnapId);
+    LOG.debug("dest snap: " + destSnapId);
     for (CompactionNode n : allNodes) {

Review Comment:
   I'll suggested to put for loop inside`LOG.isDebugEnabled()` check .
   
   ```
   if (LOG.isDebugEnabled()) {
     for (CompactionNode n : allNodes) {
       LOG.debug("Files are: " + n.fileName);
     }
   }
   ```



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -688,130 +868,139 @@ public synchronized void printMutableGraph(
       iter.remove();
       iter = topLevelNodes.iterator();
     }
-    LOG.warn("src snap:" + srcSnapId);
-    LOG.warn("dest snap:" + destSnapId);
+    LOG.debug("src snap: " + srcSnapId);
+    LOG.debug("dest snap: " + destSnapId);
     for (CompactionNode n : allNodes) {
-      LOG.warn("Files are :" + n.fileName);
+      LOG.debug("Files are: " + n.fileName);
     }
   }
 
+  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
+    return compactionDAGFwd;
+  }
 
-  public void createSnapshot(RocksDB rocksDB) throws InterruptedException {
-
-    LOG.warn("Current time is::" + System.currentTimeMillis());
-    long t1 = System.currentTimeMillis();
-
-    cpPath = cpPath + lastSnapshotCounter;
-    createCheckPoint(rocksDbPath, cpPath, rocksDB);
-    allSnapshots[lastSnapshotCounter] = new Snapshot(cpPath,
-    lastSnapshotPrefix, lastSnapshotCounter);
-
-    long t2 = System.currentTimeMillis();
-    LOG.warn("Current time is::" + t2);
-
-    LOG.warn("millisecond difference is ::" + (t2 - t1));
-   Thread.sleep(100);
-   ++lastSnapshotCounter;
-   lastSnapshotPrefix = "sid_" + lastSnapshotCounter;
-   LOG.warn("done :: 1");
+  public MutableGraph<CompactionNode> getCompactionReverseDAG() {
+    return compactionDAGReverse;
   }
 
+  /**
+   * Populate the compaction DAG with input and output SST files lists.
+   */
+  @SuppressFBWarnings({"AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION"})
+  private void populateCompactionDAG(List<String> inputFiles,
+      List<String> outputFiles, long seqNum) {
 
-  public void printAllSnapshots() throws InterruptedException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
-      }
-      LOG.warn("Snapshot id" + snap.snapshotID);
-      LOG.warn("Snapshot path" + snap.dbPath);
-      LOG.warn("Snapshot Generation" + snap.snapshotGeneration);
-      LOG.warn("");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Input {} -> Output {}", inputFiles, outputFiles);
     }
-  }
 
-  public void diffAllSnapshots() throws InterruptedException, RocksDBException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
+    for (String outfile : outputFiles) {
+      CompactionNode outfileNode = compactionNodeTable.get(outfile);
+      if (outfileNode == null) {
+        long numKeys = 0L;
+        try {
+          numKeys = getSSTFileSummary(outfile);
+        } catch (Exception e) {
+          LOG.warn("Exception in getSSTFileSummary: {}", e.getMessage());
+        }
+        outfileNode = new CompactionNode(outfile, null, numKeys, seqNum);
+        compactionDAGFwd.addNode(outfileNode);
+        compactionDAGReverse.addNode(outfileNode);
+        compactionNodeTable.put(outfile, outfileNode);
       }
-      printSnapdiffSSTFiles(allSnapshots[lastSnapshotCounter - 1], snap);
-    }
-  }
 
-  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
-    return compactionDAGFwd;
-  }
+      for (String infile : inputFiles) {
+        CompactionNode infileNode = compactionNodeTable.get(infile);
+        if (infileNode == null) {
+          long numKeys = 0L;
+          try {
+            numKeys = getSSTFileSummary(infile);
+          } catch (Exception e) {
+            LOG.warn("Exception in getSSTFileSummary: {}", e.getMessage());
+          }
+          infileNode = new CompactionNode(infile, null, numKeys, seqNum);
+          compactionDAGFwd.addNode(infileNode);
+          compactionDAGReverse.addNode(infileNode);
+          compactionNodeTable.put(infile, infileNode);
+        }
+        // Draw the edges
+        if (outfileNode.fileName.compareToIgnoreCase(
+            infileNode.fileName) != 0) {
+          compactionDAGFwd.putEdge(outfileNode, infileNode);
+          compactionDAGReverse.putEdge(infileNode, outfileNode);
+        }
+      }
+    }
 
-  public MutableGraph<CompactionNode> getCompactionReverseDAG() {
-    return compactionDAGFwd;
   }
 
+  @VisibleForTesting
   public synchronized void traverseGraph(
       MutableGraph<CompactionNode> reverseMutableGraph,
       MutableGraph<CompactionNode> fwdMutableGraph) {
 
-      List<CompactionNode> nodeList =
-        compactionNodeTable.values().stream().collect(Collectors.toList());
-    Collections.sort(nodeList, new NodeComparator());
+    List<CompactionNode> nodeList = compactionNodeTable.values().stream()
+        .sorted(new NodeComparator()).collect(Collectors.toList());
 
-    for (CompactionNode  infileNode : nodeList ) {
-      // fist go through fwdGraph to find nodes that don't have succesors.
+    for (CompactionNode infileNode : nodeList) {
+      // fist go through fwdGraph to find nodes that don't have successors.
       // These nodes will be the top level nodes in reverse graph
       Set<CompactionNode> successors = fwdMutableGraph.successors(infileNode);
-      if (successors == null || successors.size() == 0) {
-        LOG.warn("traverseGraph : No successors. cumulative " +
-            "keys : " + infileNode.cumulativeKeysReverseTraversal + "::total " +
-            "keys ::" + infileNode.totalNumberOfKeys);
+      if (successors.size() == 0) {
+        LOG.debug("No successors. Cumulative keys: {}, total keys: {}",
+            infileNode.cumulativeKeysReverseTraversal,
+            infileNode.totalNumberOfKeys);
         infileNode.cumulativeKeysReverseTraversal =
             infileNode.totalNumberOfKeys;
       }
     }
 
     HashSet<CompactionNode> visited = new HashSet<>();
-    for (CompactionNode  infileNode : nodeList ) {
+    for (CompactionNode infileNode : nodeList) {
       if (visited.contains(infileNode)) {
         continue;
       }
       visited.add(infileNode);
-      System.out.print("traverseGraph: Visiting node " + infileNode.fileName +
-          ":\n");
+      LOG.debug("Visiting node '{}'", infileNode.fileName);
       Set<CompactionNode> nextLevel = new HashSet<>();
       nextLevel.add(infileNode);
-      Set<CompactionNode> currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
+      Set<CompactionNode> currentLevel = new HashSet<>(nextLevel);
       nextLevel = new HashSet<>();
       int i = 1;
       while (currentLevel.size() != 0) {
-        LOG.warn("traverseGraph : DAG Level :" + i++);
+        LOG.debug("DAG Level {}", i++);
         for (CompactionNode current : currentLevel) {
-          LOG.warn("traverseGraph : expanding node " + current.fileName);
+          LOG.debug("Expanding node {}", current.fileName);
           Set<CompactionNode> successors =
               reverseMutableGraph.successors(current);
-          if (successors == null || successors.size() == 0) {
-            LOG.warn("traverseGraph : No successors. cumulative " +
-                "keys : " + current.cumulativeKeysReverseTraversal);
+          if (successors.size() == 0) {
+            LOG.debug("No successors. Cumulative keys: {}",
+                current.cumulativeKeysReverseTraversal);
           } else {
             for (CompactionNode oneSucc : successors) {
-              LOG.warn("traverseGraph : Adding to the next level : " +
-                  oneSucc.fileName);
-              LOG.warn("traverseGraph : " + oneSucc.fileName + "cum" + " keys"
-                  + oneSucc.cumulativeKeysReverseTraversal + "parent" + " " +
-                  current.fileName + " total " + current.totalNumberOfKeys);
+              LOG.debug("Adding to the next level: {}", oneSucc.fileName);
+              LOG.debug("'{}' cumulative keys: {}. parent '{}' total keys: {}",
+                  oneSucc.fileName, oneSucc.cumulativeKeysReverseTraversal,
+                  current.fileName, current.totalNumberOfKeys);
               oneSucc.cumulativeKeysReverseTraversal +=
                   current.cumulativeKeysReverseTraversal;
               nextLevel.add(oneSucc);
             }
           }
         }
-        currentLevel = new HashSet<>();
-        currentLevel.addAll(nextLevel);
+        currentLevel = new HashSet<>(nextLevel);
         nextLevel = new HashSet<>();
-        LOG.warn("");
       }
     }
   }
 
+  @VisibleForTesting
   public boolean debugEnabled(Integer level) {
     return DEBUG_LEVEL.contains(level);
   }
+
+  @VisibleForTesting
+  public static Logger getLog() {
+    return LOG;
+  }

Review Comment:
   Why do we need these functions?



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,134 +611,287 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  private long reconstructionSnapshotGeneration;
+
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   */
+  private synchronized void processCompactionLogLine(String line) {
+
+    LOG.debug("Processing line: {}", line);
+
+    if (line.startsWith("#")) {
+      // Skip comments
+      LOG.debug("Comment line, skipped");
+    } else if (line.startsWith(COMPACTION_LOG_SEQNUM_LINE_PREFIX)) {
+      // Read sequence number
+      LOG.debug("Reading sequence number as snapshot generation");
+      final String seqNumStr =
+          line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length()).trim();
+      // This would the snapshot generation for the nodes to come
+      reconstructionSnapshotGeneration = Long.parseLong(seqNumStr);
+    } else if (line.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) {
+      // Read compaction log entry
+
+      // Trim the beginning
+      line = line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length());
+      final String[] io = line.split(":");
+      if (io.length != 2) {
+        LOG.error("Invalid line in compaction log: {}", line);
+        return;
+      }
+      final String[] inputFiles = io[0].split(",");
+      final String[] outputFiles = io[1].split(",");
+      populateCompactionDAG(Arrays.asList(inputFiles),
+          Arrays.asList(outputFiles), reconstructionSnapshotGeneration);
+    } else {
+      LOG.error("Invalid line in compaction log: {}", line);
+    }
+  }
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> logLineStream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      logLineStream.forEach(this::processCompactionLogLine);
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Returns a set of SST nodes that doesn't exist in the in-memory DAG.
+   */
+  private Set<String> getNonExistentSSTSet(Set<String> sstSet) {
+
+    // Make a copy of sstSet
+    HashSet<String> loadSet = new HashSet<>(sstSet);
+
+    // Check if all the nodes in the provided SST set is already loaded in DAG
+    for (String sstFile : sstSet) {
+      if (compactionNodeTable.containsKey(sstFile)) {
+        loadSet.remove(sstFile);
+      }
+    }
+
+    return loadSet;
+  }
+
+  /**
+   * Returns true only when all nodes in sstSet exists in DAG.
+   */
+  private boolean isSSTSetLoaded(HashSet<String> sstSet) {
+
+    return getNonExistentSSTSet(sstSet).size() == 0;
+  }
+
+  /**
+   * Read compaction log until all dest (and src) db checkpoint SST
+   * nodes show up in the graph, or when it reaches the end of the log.
+   */
+  private boolean loadCompactionDAGBySSTSet(HashSet<String> sstSet) {
+
+    // Get a set of SSTs that doesn't exist in the current in-memory DAG
+    Set<String> loadSet = getNonExistentSSTSet(sstSet);
+
+    if (loadSet.size() == 0) {
+      // All expected nodes in the sstSet are already there,
+      //  no need to read/load any compaction log from disk.
+      return true;
+    }
+
+    // Otherwise, load compaction logs in order until all nodes are present in
+    //  the DAG.
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString()
+              .toLowerCase().endsWith(COMPACTION_LOG_FILENAME_SUFFIX))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+
+          // TODO: Potential optimization: stop reading as soon as all nodes are
+          //  there. Currently it loads an entire file at a time.
+          readCompactionLogToDAG(logPath.toString());
+
+          for (Iterator<String> it = loadSet.iterator(); it.hasNext();) {
+            String sstFile = it.next();
+            if (compactionNodeTable.containsKey(sstFile)) {
+              LOG.debug("Found SST node: {}", sstFile);
+              it.remove();
+            }
+          }
+
+          if (loadSet.size() == 0) {
+            break;
+          }
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+
+    // Just in case there are still nodes to be expected not loaded.
+    if (loadSet.size() > 0) {
+      LOG.warn("The following nodes are missing from the compaction log: {}. "
+          + "Possibly because those a newly flushed SSTs that haven't gone "
+          + "though any compaction yet", loadSet);
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Load existing compaction log files to the in-memory DAG.
+   * This only needs to be done once during OM startup.
+   */
+  public synchronized void loadAllCompactionLogs() {
+    if (compactionLogDir == null) {
+      throw new RuntimeException("Compaction log directory must be set first");
+    }
+    reconstructionSnapshotGeneration = 0L;
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase().endsWith(".log"))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+          readCompactionLogToDAG(logPath.toString());
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+  }
+
+  /**
+   * Get a list of SST files that differs between src and destination snapshots.
+   * <p>
+   * Expected input: src is a snapshot taken AFTER the dest.
+   *
+   * @param src source snapshot
+   * @param dest destination snapshot
+   */
+  public synchronized List<String> getSSTDiffList(Snapshot src, Snapshot dest) {
+
+    LOG.debug("src '{}' -> dest '{}'", src.dbPath, dest.dbPath);
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
 
     HashSet<String> fwdDAGSameFiles = new HashSet<>();
     HashSet<String> fwdDAGDifferentFiles = new HashSet<>();
 
-    LOG.warn("Doing forward diff between source and destination " +
-        "Snapshots:" + src.dbPath + ", " + dest.dbPath);
-    realPrintSnapdiffSSTFiles(src, dest, srcSnapFiles, destSnapFiles,
-        compactionDAGFwd,
-        fwdDAGSameFiles,
-        fwdDAGDifferentFiles);
+    LOG.debug("Doing forward diff between src and dest snapshots: " +
+        src.dbPath + " to " + dest.dbPath);
+    internalGetSSTDiffList(src, dest, srcSnapFiles, destSnapFiles,
+        compactionDAGFwd, fwdDAGSameFiles, fwdDAGDifferentFiles);
 
-    LOG.warn("Overall Summary \n" +
-            "Doing Overall diff between source and destination Snapshots:" +
-        src.dbPath + ", " + dest.dbPath);
-    System.out.print("fwd DAG Same files :");
-    for (String file : fwdDAGSameFiles) {
-      System.out.print(file + ", ");
-    }
-    LOG.warn("");
-    System.out.print("\nFwd DAG Different files :");
-    for (String file : fwdDAGDifferentFiles) {
-      CompactionNode n = compactionNodeTable.get(file);
-      System.out.print(file + ", ");
+    List<String> res = new ArrayList<>();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Result of diff from src '" + src.dbPath + "' to dest '" +
+          dest.dbPath + "':");
+      StringBuilder logSB = new StringBuilder();
+
+      logSB.append("Fwd DAG same SST files:      ");
+      for (String file : fwdDAGSameFiles) {
+        logSB.append(file).append(" ");
+      }
+      LOG.debug(logSB.toString());
+
+      logSB.setLength(0);
+      logSB.append("Fwd DAG different SST files: ");
+      for (String file : fwdDAGDifferentFiles) {
+        logSB.append(file).append(" ");
+        res.add(file);
+      }
+      LOG.debug(logSB.toString());
+
+    } else {
+      res.addAll(fwdDAGDifferentFiles);
     }
-    LOG.warn("");
+
+    return res;
   }
 
   @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-  public synchronized void realPrintSnapdiffSSTFiles(
+  public synchronized void internalGetSSTDiffList(
       Snapshot src, Snapshot dest,
       HashSet<String> srcSnapFiles,
       HashSet<String> destSnapFiles,
       MutableGraph<CompactionNode> mutableGraph,
       HashSet<String> sameFiles, HashSet<String> differentFiles) {
 
-
     for (String fileName : srcSnapFiles) {
       if (destSnapFiles.contains(fileName)) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + " and Dest " +
-            "Snapshot" + dest.dbPath + " Contain Same file " + fileName);
+        LOG.debug("Source '{}' and destination '{}' share the same SST '{}'",
+            src.dbPath, dest.dbPath, fileName);
         sameFiles.add(fileName);
         continue;
       }
       CompactionNode infileNode =
           compactionNodeTable.get(Paths.get(fileName).getFileName().toString());
       if (infileNode == null) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + "File " + fileName + "was " +
-            "never compacted");
+        LOG.debug("Src " + src.dbPath + " File " + fileName +
+            " was never compacted");
         differentFiles.add(fileName);
         continue;
       }
-      System.out.print(" Expandin File:" + fileName + ":\n");
-      Set<CompactionNode> nextLevel = new HashSet<>();
-      nextLevel.add(infileNode);
+      LOG.debug("Expanding SST file: " + fileName);
       Set<CompactionNode> currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
-      nextLevel = new HashSet<>();
+      currentLevel.add(infileNode);
+      Set<CompactionNode> nextLevel = new HashSet<>();

Review Comment:
   If you move `nextLevel` deceleration and initialization  inside while loop (line #850), you won't have to reset it at line #887.



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -608,52 +792,48 @@ public Comparator<CompactionNode> reversed() {
     }
   }
 
-
-  public void dumpCompactioNodeTable() {
-    List<CompactionNode> nodeList =
-        compactionNodeTable.values().stream().collect(Collectors.toList());
-    Collections.sort(nodeList, new NodeComparator());
-    for (CompactionNode n : nodeList ) {
-      LOG.warn("File : " + n.fileName + " :: Total keys : "
-          + n.totalNumberOfKeys);
-      LOG.warn("File : " + n.fileName + " :: Cumulative keys : "  +
+  @VisibleForTesting
+  public void dumpCompactionNodeTable() {
+    List<CompactionNode> nodeList = compactionNodeTable.values().stream()
+        .sorted(new NodeComparator()).collect(Collectors.toList());
+    for (CompactionNode n : nodeList) {
+      LOG.info("File '{}' total keys: {}", n.fileName, n.totalNumberOfKeys);
+      LOG.info("File '{}' cumulative keys: {}", n.fileName,
           n.cumulativeKeysReverseTraversal);
     }
   }
 
+  @VisibleForTesting
   @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-  public synchronized void printMutableGraphFromAGivenNode(
-      String fileName, int level, MutableGraph<CompactionNode> mutableGraph) {
+  public synchronized void printMutableGraphFromAGivenNode(String fileName,
+      int level, MutableGraph<CompactionNode> mutableGraph) {
     CompactionNode infileNode =
         compactionNodeTable.get(Paths.get(fileName).getFileName().toString());
     if (infileNode == null) {
       return;
     }
-    System.out.print("\nCompaction Level : " + level + " Expandin File:" +
-        fileName + ":\n");
+    LOG.info("\nCompaction Level: " + level + " Expanding File: " + fileName);
     Set<CompactionNode> nextLevel = new HashSet<>();
     nextLevel.add(infileNode);
-    Set<CompactionNode> currentLevel = new HashSet<>();
-    currentLevel.addAll(nextLevel);
+    Set<CompactionNode> currentLevel = new HashSet<>(nextLevel);

Review Comment:
   Same as previously commented.



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,134 +558,231 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   */
+  private synchronized void processCompactionLogLine(String line) {
+
+    LOG.debug("Processing line: {}", line);
+
+    if (line.startsWith("#")) {
+      // Skip comments
+      LOG.debug("Comment line, skipped");
+    } else if (line.startsWith(COMPACTION_LOG_SEQNUM_LINE_PREFIX)) {
+      // Read sequence number
+      LOG.debug("Reading sequence number as snapshot generation");
+      final String seqNumStr =
+          line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length()).trim();
+      // This would the snapshot generation for the nodes to come
+      reconstructionSnapshotGeneration = Long.parseLong(seqNumStr);
+    } else if (line.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) {
+      // Read compaction log entry
+
+      // Trim the beginning
+      line = line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length());
+      final String[] io = line.split(":");
+      if (io.length != 2) {
+        LOG.error("Invalid line in compaction log: {}", line);
+        return;
+      }
+      final String[] inputFiles = io[0].split(",");
+      final String[] outputFiles = io[1].split(",");
+      populateCompactionDAG(Arrays.asList(inputFiles),
+          Arrays.asList(outputFiles), reconstructionSnapshotGeneration);
+    } else {
+      LOG.error("Invalid line in compaction log: {}", line);
+    }
+  }
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> logLineStream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      logLineStream.forEach(this::processCompactionLogLine);
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Load existing compaction log files to the in-memory DAG.
+   * This only needs to be done once during OM startup.
+   */
+  public synchronized void loadAllCompactionLogs() {
+    if (compactionLogDir == null) {
+      throw new RuntimeException("Compaction log directory must be set first");
+    }
+    reconstructionSnapshotGeneration = 0L;
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase()
+              .endsWith(COMPACTION_LOG_FILENAME_SUFFIX))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+          readCompactionLogToDAG(logPath.toString());
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+  }
+
+  /**
+   * Snapshot information node class in the DAG.
+   */
+  static class Snapshot {
+    private final String dbPath;
+    private final String snapshotID;
+    private final long snapshotGeneration;
+
+    Snapshot(String db, String id, long gen) {
+      dbPath = db;
+      snapshotID = id;
+      snapshotGeneration = gen;
+    }
+
+    public String getDbPath() {
+      return dbPath;
+    }
+
+    public String getSnapshotID() {
+      return snapshotID;
+    }
+
+    public long getSnapshotGeneration() {
+      return snapshotGeneration;
+    }
+  }
+
+  /**
+   * Get a list of SST files that differs between src and destination snapshots.
+   * <p>
+   * Expected input: src is a snapshot taken AFTER the dest.
+   *
+   * @param src source snapshot
+   * @param dest destination snapshot
+   */
+  public synchronized List<String> getSSTDiffList(Snapshot src, Snapshot dest) {
+
+    LOG.debug("src '{}' -> dest '{}'", src.dbPath, dest.dbPath);
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
 
     HashSet<String> fwdDAGSameFiles = new HashSet<>();
     HashSet<String> fwdDAGDifferentFiles = new HashSet<>();
 
-    LOG.warn("Doing forward diff between source and destination " +
-        "Snapshots:" + src.dbPath + ", " + dest.dbPath);
-    realPrintSnapdiffSSTFiles(src, dest, srcSnapFiles, destSnapFiles,
-        compactionDAGFwd,
-        fwdDAGSameFiles,
-        fwdDAGDifferentFiles);
+    LOG.debug("Doing forward diff between src and dest snapshots: " +
+        src.dbPath + " to " + dest.dbPath);
+    internalGetSSTDiffList(src, dest, srcSnapFiles, destSnapFiles,
+        compactionDAGFwd, fwdDAGSameFiles, fwdDAGDifferentFiles);
 
-    LOG.warn("Overall Summary \n" +
-            "Doing Overall diff between source and destination Snapshots:" +
-        src.dbPath + ", " + dest.dbPath);
-    System.out.print("fwd DAG Same files :");
-    for (String file : fwdDAGSameFiles) {
-      System.out.print(file + ", ");
-    }
-    LOG.warn("");
-    System.out.print("\nFwd DAG Different files :");
-    for (String file : fwdDAGDifferentFiles) {
-      CompactionNode n = compactionNodeTable.get(file);
-      System.out.print(file + ", ");
+    List<String> res = new ArrayList<>();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Result of diff from src '" + src.dbPath + "' to dest '" +
+          dest.dbPath + "':");
+      StringBuilder logSB = new StringBuilder();
+
+      logSB.append("Fwd DAG same SST files:      ");
+      for (String file : fwdDAGSameFiles) {
+        logSB.append(file).append(" ");
+      }
+      LOG.debug(logSB.toString());
+
+      logSB.setLength(0);
+      logSB.append("Fwd DAG different SST files: ");
+      for (String file : fwdDAGDifferentFiles) {
+        logSB.append(file).append(" ");
+        res.add(file);
+      }
+      LOG.debug(logSB.toString());
+
+    } else {
+      res.addAll(fwdDAGDifferentFiles);
     }
-    LOG.warn("");
+
+    return res;
   }
 
+  /**
+   * Core getSSTDiffList logic.
+   */
   @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-  public synchronized void realPrintSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest,
-      HashSet<String> srcSnapFiles,
-      HashSet<String> destSnapFiles,
+  private void internalGetSSTDiffList(Snapshot src, Snapshot dest,
+      HashSet<String> srcSnapFiles, HashSet<String> destSnapFiles,
       MutableGraph<CompactionNode> mutableGraph,
       HashSet<String> sameFiles, HashSet<String> differentFiles) {
 
-
     for (String fileName : srcSnapFiles) {
       if (destSnapFiles.contains(fileName)) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + " and Dest " +
-            "Snapshot" + dest.dbPath + " Contain Same file " + fileName);
+        LOG.debug("Source '{}' and destination '{}' share the same SST '{}'",
+            src.dbPath, dest.dbPath, fileName);
         sameFiles.add(fileName);
         continue;
       }
       CompactionNode infileNode =
           compactionNodeTable.get(Paths.get(fileName).getFileName().toString());
       if (infileNode == null) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + "File " + fileName + "was " +
-            "never compacted");
+        LOG.debug("Src " + src.dbPath + " File " + fileName +
+            " was never compacted");
         differentFiles.add(fileName);
         continue;
       }
-      System.out.print(" Expandin File:" + fileName + ":\n");
-      Set<CompactionNode> nextLevel = new HashSet<>();
-      nextLevel.add(infileNode);
+      LOG.debug("Expanding SST file: " + fileName);
       Set<CompactionNode> currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
-      nextLevel = new HashSet<>();
+      currentLevel.add(infileNode);
+      Set<CompactionNode> nextLevel = new HashSet<>();
       int i = 1;
       while (currentLevel.size() != 0) {
-        LOG.warn("DAG Level :" + i++);
+        LOG.debug("DAG Level: " + i++);
         for (CompactionNode current : currentLevel) {
-          LOG.warn("acknowledging file " + current.fileName);
+          LOG.debug("Acknowledging file " + current.fileName);
           if (current.snapshotGeneration <= dest.snapshotGeneration) {
-            LOG.warn("Reached dest generation count. SrcSnapshot : " +
-                src.dbPath + " and Dest " + "Snapshot" + dest.dbPath +
-                " Contain Diffrent file " + current.fileName);
+            LOG.debug("Reached dest generation count. Src: " +
+                src.dbPath + " and Dest: " + dest.dbPath +
+                " have different file: " + current.fileName);
             differentFiles.add(current.fileName);
             continue;
           }
           Set<CompactionNode> successors = mutableGraph.successors(current);
-          if (successors == null || successors.size() == 0) {
-            LOG.warn("No further compaction for the file" +
-                ".SrcSnapshot : " + src.dbPath + " and Dest " +
-                "Snapshot" + dest.dbPath + " Contain Diffrent file " +
-                current.fileName);
+          if (successors.size() == 0) {
+            LOG.debug("No further compaction happened for the current file. " +
+                "src: " + src.dbPath + " and dest: " + dest.dbPath +
+                " have different file: " + current.fileName);
             differentFiles.add(current.fileName);
           } else {
             for (CompactionNode oneSucc : successors) {
               if (sameFiles.contains(oneSucc.fileName) ||
                   differentFiles.contains(oneSucc.fileName)) {
-                LOG.warn("Skipping file :" + oneSucc.fileName);
+                LOG.debug("Skipping known same file: " + oneSucc.fileName);
                 continue;
               }
               if (destSnapFiles.contains(oneSucc.fileName)) {
-                LOG.warn("SrcSnapshot : " + src.dbPath + " and Dest " +
-                    "Snapshot" + dest.dbPath + " Contain Same file " +
-                    oneSucc.fileName);
+                LOG.debug("src: " + src.dbPath + " and dest: " + dest.dbPath +
+                    " have the same file: " + oneSucc.fileName);
                 sameFiles.add(oneSucc.fileName);
                 continue;
               } else {

Review Comment:
   `continue` here doesn't make any difference. May be you want to do something like:
   
   ```
   if (sameFiles.contains(oneSucc.fileName) ||
       differentFiles.contains(oneSucc.fileName)) {
     LOG.debug("Skipping known same file: " + oneSucc.fileName);
     continue;
   }
   
   if (destSnapFiles.contains(oneSucc.fileName)) {
     LOG.debug("src: " + src.dbPath + " and dest: " + dest.dbPath +
        " have the same file: " + oneSucc.fileName);
     sameFiles.add(oneSucc.fileName);
     continue;
   }
   
   LOG.debug("src " + src.dbPath + " and dest " + dest.dbPath +
       " have a different SST: " + oneSucc.fileName);
   nextLevel.add(oneSucc);  
   ```



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -471,134 +558,231 @@ public HashSet<String> readRocksDBLiveFiles(String dbPathArg) {
     return liveFiles;
   }
 
-  // Given the src and destination Snapshots, it prints a Diff list.
-  private synchronized void printSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest) throws RocksDBException {
-    LOG.warn("Src Snapshot files :" + src.dbPath);
+  /**
+   * Process each line of compaction log text file input and populate the DAG.
+   */
+  private synchronized void processCompactionLogLine(String line) {
+
+    LOG.debug("Processing line: {}", line);
+
+    if (line.startsWith("#")) {
+      // Skip comments
+      LOG.debug("Comment line, skipped");
+    } else if (line.startsWith(COMPACTION_LOG_SEQNUM_LINE_PREFIX)) {
+      // Read sequence number
+      LOG.debug("Reading sequence number as snapshot generation");
+      final String seqNumStr =
+          line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length()).trim();
+      // This would the snapshot generation for the nodes to come
+      reconstructionSnapshotGeneration = Long.parseLong(seqNumStr);
+    } else if (line.startsWith(COMPACTION_LOG_ENTRY_LINE_PREFIX)) {
+      // Read compaction log entry
+
+      // Trim the beginning
+      line = line.substring(COMPACTION_LOG_SEQNUM_LINE_PREFIX.length());
+      final String[] io = line.split(":");
+      if (io.length != 2) {
+        LOG.error("Invalid line in compaction log: {}", line);
+        return;
+      }
+      final String[] inputFiles = io[0].split(",");
+      final String[] outputFiles = io[1].split(",");
+      populateCompactionDAG(Arrays.asList(inputFiles),
+          Arrays.asList(outputFiles), reconstructionSnapshotGeneration);
+    } else {
+      LOG.error("Invalid line in compaction log: {}", line);
+    }
+  }
+
+  /**
+   * Helper to read compaction log to the internal DAG.
+   */
+  private void readCompactionLogToDAG(String currCompactionLogPath) {
+    LOG.debug("Loading compaction log: {}", currCompactionLogPath);
+    try (Stream<String> logLineStream =
+        Files.lines(Paths.get(currCompactionLogPath), StandardCharsets.UTF_8)) {
+      logLineStream.forEach(this::processCompactionLogLine);
+    } catch (IOException ioEx) {
+      throw new RuntimeException(ioEx);
+    }
+  }
+
+  /**
+   * Load existing compaction log files to the in-memory DAG.
+   * This only needs to be done once during OM startup.
+   */
+  public synchronized void loadAllCompactionLogs() {
+    if (compactionLogDir == null) {
+      throw new RuntimeException("Compaction log directory must be set first");
+    }
+    reconstructionSnapshotGeneration = 0L;
+    try {
+      try (Stream<Path> pathStream = Files.list(Paths.get(compactionLogDir))
+          .filter(e -> e.toString().toLowerCase()
+              .endsWith(COMPACTION_LOG_FILENAME_SUFFIX))
+          .sorted()) {
+        for (Path logPath : pathStream.collect(Collectors.toList())) {
+          readCompactionLogToDAG(logPath.toString());
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException("Error listing compaction log dir " +
+          compactionLogDir, e);
+    }
+  }
+
+  /**
+   * Snapshot information node class in the DAG.
+   */
+  static class Snapshot {
+    private final String dbPath;
+    private final String snapshotID;
+    private final long snapshotGeneration;
+
+    Snapshot(String db, String id, long gen) {
+      dbPath = db;
+      snapshotID = id;
+      snapshotGeneration = gen;
+    }
+
+    public String getDbPath() {
+      return dbPath;
+    }
+
+    public String getSnapshotID() {
+      return snapshotID;
+    }
+
+    public long getSnapshotGeneration() {
+      return snapshotGeneration;
+    }
+  }
+
+  /**
+   * Get a list of SST files that differs between src and destination snapshots.
+   * <p>
+   * Expected input: src is a snapshot taken AFTER the dest.
+   *
+   * @param src source snapshot
+   * @param dest destination snapshot
+   */
+  public synchronized List<String> getSSTDiffList(Snapshot src, Snapshot dest) {
+
+    LOG.debug("src '{}' -> dest '{}'", src.dbPath, dest.dbPath);
     HashSet<String> srcSnapFiles = readRocksDBLiveFiles(src.dbPath);
-    LOG.warn("dest Snapshot files :" + dest.dbPath);
     HashSet<String> destSnapFiles = readRocksDBLiveFiles(dest.dbPath);
 
     HashSet<String> fwdDAGSameFiles = new HashSet<>();
     HashSet<String> fwdDAGDifferentFiles = new HashSet<>();
 
-    LOG.warn("Doing forward diff between source and destination " +
-        "Snapshots:" + src.dbPath + ", " + dest.dbPath);
-    realPrintSnapdiffSSTFiles(src, dest, srcSnapFiles, destSnapFiles,
-        compactionDAGFwd,
-        fwdDAGSameFiles,
-        fwdDAGDifferentFiles);
+    LOG.debug("Doing forward diff between src and dest snapshots: " +
+        src.dbPath + " to " + dest.dbPath);
+    internalGetSSTDiffList(src, dest, srcSnapFiles, destSnapFiles,
+        compactionDAGFwd, fwdDAGSameFiles, fwdDAGDifferentFiles);
 
-    LOG.warn("Overall Summary \n" +
-            "Doing Overall diff between source and destination Snapshots:" +
-        src.dbPath + ", " + dest.dbPath);
-    System.out.print("fwd DAG Same files :");
-    for (String file : fwdDAGSameFiles) {
-      System.out.print(file + ", ");
-    }
-    LOG.warn("");
-    System.out.print("\nFwd DAG Different files :");
-    for (String file : fwdDAGDifferentFiles) {
-      CompactionNode n = compactionNodeTable.get(file);
-      System.out.print(file + ", ");
+    List<String> res = new ArrayList<>();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Result of diff from src '" + src.dbPath + "' to dest '" +
+          dest.dbPath + "':");
+      StringBuilder logSB = new StringBuilder();
+
+      logSB.append("Fwd DAG same SST files:      ");
+      for (String file : fwdDAGSameFiles) {
+        logSB.append(file).append(" ");
+      }
+      LOG.debug(logSB.toString());
+
+      logSB.setLength(0);
+      logSB.append("Fwd DAG different SST files: ");
+      for (String file : fwdDAGDifferentFiles) {
+        logSB.append(file).append(" ");
+        res.add(file);
+      }
+      LOG.debug(logSB.toString());
+
+    } else {
+      res.addAll(fwdDAGDifferentFiles);
     }
-    LOG.warn("");
+
+    return res;
   }
 
+  /**
+   * Core getSSTDiffList logic.
+   */
   @SuppressFBWarnings({"NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE"})
-  public synchronized void realPrintSnapdiffSSTFiles(
-      Snapshot src, Snapshot dest,
-      HashSet<String> srcSnapFiles,
-      HashSet<String> destSnapFiles,
+  private void internalGetSSTDiffList(Snapshot src, Snapshot dest,
+      HashSet<String> srcSnapFiles, HashSet<String> destSnapFiles,
       MutableGraph<CompactionNode> mutableGraph,
       HashSet<String> sameFiles, HashSet<String> differentFiles) {
 
-
     for (String fileName : srcSnapFiles) {
       if (destSnapFiles.contains(fileName)) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + " and Dest " +
-            "Snapshot" + dest.dbPath + " Contain Same file " + fileName);
+        LOG.debug("Source '{}' and destination '{}' share the same SST '{}'",
+            src.dbPath, dest.dbPath, fileName);
         sameFiles.add(fileName);
         continue;
       }
       CompactionNode infileNode =
           compactionNodeTable.get(Paths.get(fileName).getFileName().toString());
       if (infileNode == null) {
-        LOG.warn("SrcSnapshot : " + src.dbPath + "File " + fileName + "was " +
-            "never compacted");
+        LOG.debug("Src " + src.dbPath + " File " + fileName +
+            " was never compacted");
         differentFiles.add(fileName);
         continue;
       }
-      System.out.print(" Expandin File:" + fileName + ":\n");
-      Set<CompactionNode> nextLevel = new HashSet<>();
-      nextLevel.add(infileNode);
+      LOG.debug("Expanding SST file: " + fileName);
       Set<CompactionNode> currentLevel = new HashSet<>();
-      currentLevel.addAll(nextLevel);
-      nextLevel = new HashSet<>();
+      currentLevel.add(infileNode);
+      Set<CompactionNode> nextLevel = new HashSet<>();
       int i = 1;
       while (currentLevel.size() != 0) {
-        LOG.warn("DAG Level :" + i++);
+        LOG.debug("DAG Level: " + i++);
         for (CompactionNode current : currentLevel) {
-          LOG.warn("acknowledging file " + current.fileName);
+          LOG.debug("Acknowledging file " + current.fileName);
           if (current.snapshotGeneration <= dest.snapshotGeneration) {
-            LOG.warn("Reached dest generation count. SrcSnapshot : " +
-                src.dbPath + " and Dest " + "Snapshot" + dest.dbPath +
-                " Contain Diffrent file " + current.fileName);
+            LOG.debug("Reached dest generation count. Src: " +
+                src.dbPath + " and Dest: " + dest.dbPath +
+                " have different file: " + current.fileName);
             differentFiles.add(current.fileName);
             continue;
           }
           Set<CompactionNode> successors = mutableGraph.successors(current);
-          if (successors == null || successors.size() == 0) {
-            LOG.warn("No further compaction for the file" +
-                ".SrcSnapshot : " + src.dbPath + " and Dest " +
-                "Snapshot" + dest.dbPath + " Contain Diffrent file " +
-                current.fileName);
+          if (successors.size() == 0) {
+            LOG.debug("No further compaction happened for the current file. " +
+                "src: " + src.dbPath + " and dest: " + dest.dbPath +
+                " have different file: " + current.fileName);
             differentFiles.add(current.fileName);
           } else {
             for (CompactionNode oneSucc : successors) {
               if (sameFiles.contains(oneSucc.fileName) ||
                   differentFiles.contains(oneSucc.fileName)) {
-                LOG.warn("Skipping file :" + oneSucc.fileName);
+                LOG.debug("Skipping known same file: " + oneSucc.fileName);
                 continue;
               }
               if (destSnapFiles.contains(oneSucc.fileName)) {
-                LOG.warn("SrcSnapshot : " + src.dbPath + " and Dest " +
-                    "Snapshot" + dest.dbPath + " Contain Same file " +
-                    oneSucc.fileName);
+                LOG.debug("src: " + src.dbPath + " and dest: " + dest.dbPath +
+                    " have the same file: " + oneSucc.fileName);
                 sameFiles.add(oneSucc.fileName);
                 continue;
               } else {
-                LOG.warn("SrcSnapshot : " + src.dbPath + " and Dest " +
-                    "Snapshot" + dest.dbPath + " Contain Diffrent file " +
-                    oneSucc.fileName);
+                LOG.debug("src " + src.dbPath + " and dest " + dest.dbPath +
+                    " have a different SST: " + oneSucc.fileName);
                 nextLevel.add(oneSucc);
               }
             }
           }
         }
-        currentLevel = new HashSet<>();
-        currentLevel.addAll(nextLevel);
+        currentLevel = new HashSet<>(nextLevel);

Review Comment:
   I don't think you need to initialize new HashSet here since it is getting reset altogether in next time. I feel it is unnecessary.



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -688,130 +868,139 @@ public synchronized void printMutableGraph(
       iter.remove();
       iter = topLevelNodes.iterator();
     }
-    LOG.warn("src snap:" + srcSnapId);
-    LOG.warn("dest snap:" + destSnapId);
+    LOG.debug("src snap: " + srcSnapId);
+    LOG.debug("dest snap: " + destSnapId);
     for (CompactionNode n : allNodes) {
-      LOG.warn("Files are :" + n.fileName);
+      LOG.debug("Files are: " + n.fileName);
     }
   }
 
+  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
+    return compactionDAGFwd;
+  }
 
-  public void createSnapshot(RocksDB rocksDB) throws InterruptedException {
-
-    LOG.warn("Current time is::" + System.currentTimeMillis());
-    long t1 = System.currentTimeMillis();
-
-    cpPath = cpPath + lastSnapshotCounter;
-    createCheckPoint(rocksDbPath, cpPath, rocksDB);
-    allSnapshots[lastSnapshotCounter] = new Snapshot(cpPath,
-    lastSnapshotPrefix, lastSnapshotCounter);
-
-    long t2 = System.currentTimeMillis();
-    LOG.warn("Current time is::" + t2);
-
-    LOG.warn("millisecond difference is ::" + (t2 - t1));
-   Thread.sleep(100);
-   ++lastSnapshotCounter;
-   lastSnapshotPrefix = "sid_" + lastSnapshotCounter;
-   LOG.warn("done :: 1");
+  public MutableGraph<CompactionNode> getCompactionReverseDAG() {
+    return compactionDAGReverse;
   }
 
+  /**
+   * Populate the compaction DAG with input and output SST files lists.
+   */
+  @SuppressFBWarnings({"AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION"})
+  private void populateCompactionDAG(List<String> inputFiles,
+      List<String> outputFiles, long seqNum) {
 
-  public void printAllSnapshots() throws InterruptedException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
-      }
-      LOG.warn("Snapshot id" + snap.snapshotID);
-      LOG.warn("Snapshot path" + snap.dbPath);
-      LOG.warn("Snapshot Generation" + snap.snapshotGeneration);
-      LOG.warn("");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Input {} -> Output {}", inputFiles, outputFiles);
     }
-  }
 
-  public void diffAllSnapshots() throws InterruptedException, RocksDBException {
-    for (Snapshot snap : allSnapshots) {
-      if (snap == null) {
-        break;
+    for (String outfile : outputFiles) {
+      CompactionNode outfileNode = compactionNodeTable.get(outfile);
+      if (outfileNode == null) {
+        long numKeys = 0L;
+        try {
+          numKeys = getSSTFileSummary(outfile);
+        } catch (Exception e) {
+          LOG.warn("Exception in getSSTFileSummary: {}", e.getMessage());
+        }
+        outfileNode = new CompactionNode(outfile, null, numKeys, seqNum);
+        compactionDAGFwd.addNode(outfileNode);
+        compactionDAGReverse.addNode(outfileNode);
+        compactionNodeTable.put(outfile, outfileNode);
       }
-      printSnapdiffSSTFiles(allSnapshots[lastSnapshotCounter - 1], snap);
-    }
-  }
 
-  public MutableGraph<CompactionNode> getCompactionFwdDAG() {
-    return compactionDAGFwd;
-  }
+      for (String infile : inputFiles) {
+        CompactionNode infileNode = compactionNodeTable.get(infile);

Review Comment:
   Code, from line #898-#910 and #913-#925, is same. You can extract out the common code to helper function.
   
   ```
     public CompactionNode getCompactionNode(String file) {
       CompactionNode fileNode = compactionNodeTable.get(file);
       if (fileNode != null) {
   
         long numKeys = 0L;
         try {
           numKeys = getSSTFileSummary(outfile);
         } catch (Exception e) {
           LOG.warn("Exception in getSSTFileSummary: {}", e.getMessage());
         }
   
         fileNode = new CompactionNode(file, null, numKeys, seqNum);
         compactionDAGFwd.addNode(fileNode);
         compactionDAGReverse.addNode(fileNode);
         compactionNodeTable.put(file, fileNode);
       }
       return fileNode;
     }
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org