jojochuang commented on code in PR #9268:
URL: https://github.com/apache/ozone/pull/9268#discussion_r2524775922


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -791,30 +741,26 @@ private void preconditionChecksForLoadAllCompactionLogs() 
{
    * exist in backup directory before being involved in compactions),
    * and appends the extension '.sst'.
    */
-  private String getSSTFullPath(String sstFilenameWithoutExtension,
-      String... dbPaths) {
+  private String getSSTFullPath(String sstFilenameWithoutExtension, Path... 
dbPaths) {

Review Comment:
   the update to this helper method is not relevant to this PR and it's 
semantically the same as before.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -350,37 +361,34 @@ private void deleteDir(Path path) {
   /**
    * Convert from SnapshotInfo to DifferSnapshotInfo.
    */
-  private DifferSnapshotInfo getDSIFromSI(SnapshotInfo snapshotInfo,
-      OmSnapshot omSnapshot, final String volumeName, final String bucketName)
-      throws IOException {
-
-    final OMMetadataManager snapshotOMMM = omSnapshot.getMetadataManager();
-    final String checkpointPath =
-        snapshotOMMM.getStore().getDbLocation().getPath();
+  private static DifferSnapshotInfo getDSIFromSI(OMMetadataManager 
activeOmMetadataManager,
+      SnapshotInfo snapshotInfo, OmSnapshotLocalData snapshotLocalData) throws 
IOException {
     final UUID snapshotId = snapshotInfo.getSnapshotId();
     final long dbTxSequenceNumber = snapshotInfo.getDbTxSequenceNumber();
+    NavigableMap<Integer, List<SstFileInfo>> versionSstFiles = 
snapshotLocalData.getVersionSstFileInfos()
+        .entrySet().stream().collect(toMap(Map.Entry::getKey, entry -> 
entry.getValue().getSstFiles(),
+            (u, v) -> {
+            throw new IllegalStateException(String.format("Duplicate key %s", 
u));
+          }, TreeMap::new));
+    if (versionSstFiles.isEmpty()) {
+      throw new IOException(String.format("No versions found corresponding to 
%s", snapshotId));
+    }
     return new DifferSnapshotInfo(
-        checkpointPath,
-        snapshotId,
-        dbTxSequenceNumber,
-        snapshotOMMM.getTableBucketPrefix(volumeName, bucketName),
-        ((RDBStore)snapshotOMMM.getStore()).getDb().getManagedRocksDb());
+        version -> OmSnapshotManager.getSnapshotPath(activeOmMetadataManager, 
snapshotId, version),
+        snapshotId, dbTxSequenceNumber, versionSstFiles);
   }
 
   @VisibleForTesting
-  protected Set<String> getSSTFileListForSnapshot(OmSnapshot snapshot,
-                                                  Set<String> tablesToLookUp) {
-    return RdbUtil.getSSTFilesForComparison(((RDBStore)snapshot
-        .getMetadataManager().getStore()).getDb().getManagedRocksDb(),
-        tablesToLookUp);
+  protected Set<SstFileInfo> getSSTFileListForSnapshot(OmSnapshot snapshot, 
Set<String> tablesToLookUp) {

Review Comment:
   it's actually a set, not a list.



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDiffUtils.java:
##########
@@ -92,95 +83,40 @@ public static Stream<Arguments> values() {
 
   @ParameterizedTest
   @MethodSource("values")
-  public void testFilterRelevantSstFilesWithPreExistingCompactionInfo(String 
validSSTColumnFamilyName,
-                                                                      String 
invalidColumnFamilyName,
-                                                                      String 
validSSTFileStartRange,
-                                                                      String 
validSSTFileEndRange,
-                                                                      String 
invalidSSTFileStartRange,
-                                                                      String 
invalidSSTFileEndRange) {
+  public void testFilterRelevantSstFilesMap(String validSSTColumnFamilyName, 
String invalidColumnFamilyName,
+      String validSSTFileStartRange, String validSSTFileEndRange, String 
invalidSSTFileStartRange,
+      String invalidSSTFileEndRange) {
     String validSstFile = "filePath/validSSTFile.sst";
     String invalidSstFile = "filePath/invalidSSTFile.sst";
     String untrackedSstFile = "filePath/untrackedSSTFile.sst";
     String expectedPrefix = 
String.valueOf((char)(((int)validSSTFileEndRange.charAt(0) +
         validSSTFileStartRange.charAt(0)) / 2));
-    Set<String> sstFile = Sets.newTreeSet(validSstFile, invalidSstFile, 
untrackedSstFile);
-    Set<String> inputSstFiles = new HashSet<>();
+    Map<String, SstFileInfo> sstFile = ImmutableMap.of(
+        validSstFile, new SstFileInfo(validSstFile, validSSTFileStartRange, 
validSSTFileEndRange,
+            validSSTColumnFamilyName), invalidSstFile, new 
SstFileInfo(invalidSstFile, invalidSSTFileStartRange,
+            invalidSSTFileEndRange, invalidColumnFamilyName), untrackedSstFile,
+        new SstFileInfo(untrackedSstFile, null, null, null));
+    Map<String, SstFileInfo> inputSstFiles = new HashMap<>();
     List<Set<String>> tablesToLookupSet = 
Arrays.asList(ImmutableSet.of(validSSTColumnFamilyName),
         ImmutableSet.of(invalidColumnFamilyName), 
ImmutableSet.of(validSSTColumnFamilyName, invalidColumnFamilyName),
         Collections.emptySet());
     for (Set<String> tablesToLookup : tablesToLookupSet) {
       inputSstFiles.clear();
-      inputSstFiles.addAll(sstFile);
+      inputSstFiles.putAll(sstFile);
       RocksDiffUtils.filterRelevantSstFiles(inputSstFiles,
+          tablesToLookup,
           new TablePrefixInfo(
               new HashMap<String, String>() {{
                 put(invalidColumnFamilyName, 
getLexicographicallyHigherString(invalidSSTFileEndRange));
                 put(validSSTColumnFamilyName, expectedPrefix);
-              }}), ImmutableMap.of("validSSTFile", new 
CompactionNode(validSstFile, 0, validSSTFileStartRange,
-                  validSSTFileEndRange, validSSTColumnFamilyName), 
"invalidSSTFile",
-                new CompactionNode(invalidSstFile, 0, invalidSSTFileStartRange,
-                  invalidSSTFileEndRange, invalidColumnFamilyName)), 
tablesToLookup);
+              }}));
       if (tablesToLookup.contains(validSSTColumnFamilyName)) {
-        Assertions.assertEquals(Sets.newTreeSet(validSstFile, 
untrackedSstFile), inputSstFiles,
+        Assertions.assertEquals(Sets.newTreeSet(validSstFile, 
untrackedSstFile), inputSstFiles.keySet(),
             "Failed for " + tablesToLookup);
       } else {
-        Assertions.assertEquals(Sets.newTreeSet(untrackedSstFile), 
inputSstFiles, "Failed for " + tablesToLookup);
-      }
-    }
-  }
-
-  private LiveFileMetaData getMockedLiveFileMetadata(String columnFamilyName, 
String startRange,
-                                                     String endRange,
-                                                     String name) {
-    LiveFileMetaData liveFileMetaData = Mockito.mock(LiveFileMetaData.class);
-    
Mockito.when(liveFileMetaData.largestKey()).thenReturn(endRange.getBytes(StandardCharsets.UTF_8));
-    
Mockito.when(liveFileMetaData.columnFamilyName()).thenReturn(columnFamilyName.getBytes(StandardCharsets.UTF_8));
-    
Mockito.when(liveFileMetaData.smallestKey()).thenReturn(startRange.getBytes(StandardCharsets.UTF_8));
-    Mockito.when(liveFileMetaData.fileName()).thenReturn("basePath/" + name + 
".sst");
-    return liveFileMetaData;
-  }
-
-  @ParameterizedTest
-  @MethodSource("values")
-  public void testFilterRelevantSstFilesFromDB(String validSSTColumnFamilyName,

Review Comment:
   why remove the test case?



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########


Review Comment:
   please update javadocs.



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -863,53 +816,109 @@ public synchronized Optional<List<String>> 
getSSTDiffListWithFullPath(DifferSnap
    *                       must be non-null.
    * @return A list of SST files without extension. e.g. ["000050", "000060"]

Review Comment:
   please update javadoc



##########
hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java:
##########
@@ -780,57 +803,31 @@ public void testGetSSTDiffListWithoutDB(String 
description,
     }
 
     // Check same and different SST files result
-    assertEquals(expectedSameSstFiles, actualSameSstFiles);
-    assertEquals(expectedDiffSstFiles, actualDiffSstFiles);
-    try (MockedStatic<RdbUtil> mockedHandler = 
Mockito.mockStatic(RdbUtil.class, Mockito.CALLS_REAL_METHODS)) {
-      RocksDB rocksDB = Mockito.mock(RocksDB.class);
-      Mockito.when(rocksDB.getName()).thenReturn("dummy");
-      Mockito.when(srcSnapshot.getRocksDB().get()).thenReturn(rocksDB);
-      Mockito.when(destSnapshot.getRocksDB().get()).thenReturn(rocksDB);
-      Mockito.when(srcSnapshot.getRocksDB().getLiveMetadataForSSTFiles())
-          .thenAnswer(invocation -> 
srcSnapshotSstFiles.stream().filter(metaDataMap::containsKey).map(file -> {
-            LiveFileMetaData liveFileMetaData = 
Mockito.mock(LiveFileMetaData.class);
-            String[] metaData = metaDataMap.get(file);
-            Mockito.when(liveFileMetaData.fileName()).thenReturn("/" + file + 
SST_FILE_EXTENSION);
-            
Mockito.when(liveFileMetaData.smallestKey()).thenReturn(metaData[0].getBytes(UTF_8));
-            
Mockito.when(liveFileMetaData.largestKey()).thenReturn(metaData[1].getBytes(UTF_8));
-            
Mockito.when(liveFileMetaData.columnFamilyName()).thenReturn(metaData[2].getBytes(UTF_8));
-            return liveFileMetaData;
-          }).collect(Collectors.toMap(liveFileMetaData -> 
FilenameUtils.getBaseName(liveFileMetaData.fileName()),
-              Function.identity())));
-      Set<String> tablesToLookup;
-      String dummyTable;
-      if (srcSnapshot.getTablePrefixes() != null) {
-        tablesToLookup = srcSnapshot.getTablePrefixes().getTableNames();
-        dummyTable = tablesToLookup.stream().findAny().get();
+    assertEquals(expectedSameSstFiles, actualSameSstFiles.keySet());
+    assertEquals(expectedDiffSstFiles, actualDiffSstFiles.keySet());
+    when(srcSnapshot.getSstFiles(eq(0), eq(tablesToLookup)))
+        .thenAnswer(invocation -> srcSnapshotSstFiles.stream()
+            .map(file -> metaDataMap.getOrDefault(file, new SstFileInfo(file, 
null, null, null)))
+            .collect(Collectors.toList()));
+    when(destSnapshot.getSstFiles(eq(0), eq(tablesToLookup)))
+        .thenAnswer(invocation -> destSnapshotSstFiles.stream()
+            .map(file -> metaDataMap.getOrDefault(file, new SstFileInfo(file, 
null, null, null)))
+            .collect(Collectors.toList()));
+
+    try {
+      Assertions.assertEquals(Optional.ofNullable(expectedSSTDiffFiles)
+              .map(files -> 
files.stream().sorted().collect(Collectors.toList())).orElse(null),
+          rocksDBCheckpointDiffer.getSSTDiffList(
+                  new DifferSnapshotVersion(srcSnapshot, 0, tablesToLookup),
+                  new DifferSnapshotVersion(destSnapshot, 0, tablesToLookup), 
prefixInfo, tablesToLookup,
+                  true)
+              .map(i -> 
i.stream().map(SstFileInfo::getFileName).sorted().collect(Collectors.toList())).orElse(null));
+    } catch (RuntimeException rtEx) {
+      if (!expectingException) {
+        rtEx.printStackTrace();

Review Comment:
   do not print to stdout.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

Reply via email to