From 853d6572a874027467653378d218a715e0cfa201 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran <47532440+swamirishi@users.noreply.github.com> Date: Thu, 12 Dec 2024 14:41:19 -0800 Subject: [PATCH] HDDS-11893. Fix full snapshot diff fallback logic because of DAG pruning (#7549) --- .../rocksdiff/RocksDBCheckpointDiffer.java | 27 ++-- .../TestRocksDBCheckpointDiffer.java | 117 ++++++++++++++++-- .../hadoop/ozone/freon/TestOMSnapshotDAG.java | 15 +-- .../om/snapshot/SnapshotDiffManager.java | 21 ++-- .../om/snapshot/TestSnapshotDiffManager.java | 29 +++-- 5 files changed, 162 insertions(+), 47 deletions(-) diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java index 08a013fc7c7..de637c93263 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java @@ -27,6 +27,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -640,7 +641,7 @@ private String trimSSTFilename(String filename) { * @param rocksDB open rocksDB instance. * @return a list of SST files (without extension) in the DB. */ - public HashSet readRocksDBLiveFiles(ManagedRocksDB rocksDB) { + public Set readRocksDBLiveFiles(ManagedRocksDB rocksDB) { HashSet liveFiles = new HashSet<>(); final List cfs = Arrays.asList( @@ -825,15 +826,15 @@ private String getSSTFullPath(String sstFilenameWithoutExtension, * e.g. ["/path/to/sstBackupDir/000050.sst", * "/path/to/sstBackupDir/000060.sst"] */ - public synchronized List getSSTDiffListWithFullPath( + public synchronized Optional> getSSTDiffListWithFullPath( DifferSnapshotInfo src, DifferSnapshotInfo dest, String sstFilesDirForSnapDiffJob ) throws IOException { - List sstDiffList = getSSTDiffList(src, dest); + Optional> sstDiffList = getSSTDiffList(src, dest); - return sstDiffList.stream() + return sstDiffList.map(diffList -> diffList.stream() .map( sst -> { String sstFullPath = getSSTFullPath(sst, src.getDbPath()); @@ -843,7 +844,7 @@ public synchronized List getSSTDiffListWithFullPath( createLink(link, srcFile); return link.toString(); }) - .collect(Collectors.toList()); + .collect(Collectors.toList())); } /** @@ -857,10 +858,8 @@ public synchronized List getSSTDiffListWithFullPath( * @param dest destination snapshot * @return A list of SST files without extension. e.g. ["000050", "000060"] */ - public synchronized List getSSTDiffList( - DifferSnapshotInfo src, - DifferSnapshotInfo dest - ) throws IOException { + public synchronized Optional> getSSTDiffList(DifferSnapshotInfo src, + DifferSnapshotInfo dest) throws IOException { // TODO: Reject or swap if dest is taken after src, once snapshot chain // integration is done. @@ -894,12 +893,18 @@ public synchronized List getSSTDiffList( LOG.debug("{}", logSB); } + // Check if the DAG traversal was able to reach all the destination SST files. + for (String destSnapFile : destSnapFiles) { + if (!fwdDAGSameFiles.contains(destSnapFile) && !fwdDAGDifferentFiles.contains(destSnapFile)) { + return Optional.empty(); + } + } + if (src.getTablePrefixes() != null && !src.getTablePrefixes().isEmpty()) { filterRelevantSstFilesFullPath(fwdDAGDifferentFiles, src.getTablePrefixes()); } - - return new ArrayList<>(fwdDAGDifferentFiles); + return Optional.of(new ArrayList<>(fwdDAGDifferentFiles)); } /** diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java index 0164e3a23bd..1e5302f2490 100644 --- a/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java +++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/test/java/org/apache/ozone/rocksdiff/TestRocksDBCheckpointDiffer.java @@ -38,6 +38,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; @@ -67,14 +68,18 @@ import org.apache.hadoop.ozone.lock.BootstrapStateHandler; import org.apache.ozone.compaction.log.CompactionFileInfo; import org.apache.ozone.compaction.log.CompactionLogEntry; +import org.apache.ozone.rocksdb.util.RdbUtil; import org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.NodeComparator; import org.apache.ozone.test.GenericTestUtils; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.MockedStatic; +import org.mockito.Mockito; import org.rocksdb.ColumnFamilyDescriptor; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.LiveFileMetaData; @@ -101,6 +106,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -231,6 +237,29 @@ public void cleanUp() { } } + private static List getPrunedCompactionEntries(boolean prune) { + List entries = new ArrayList<>(); + if (!prune) { + entries.add(createCompactionEntry(1, + now(), + Arrays.asList("1", "2"), + Arrays.asList("4", "5"))); + } + entries.addAll(Arrays.asList(createCompactionEntry(2, + now(), + Arrays.asList("4", "5"), + Collections.singletonList("10")), + createCompactionEntry(3, + now(), + Arrays.asList("3", "13", "14"), + Arrays.asList("6", "7")), + createCompactionEntry(4, + now(), + Arrays.asList("6", "7"), + Collections.singletonList("11")))); + return entries; + } + /** * Test cases for testGetSSTDiffListWithoutDB. */ @@ -306,13 +335,17 @@ private static Stream casesGetSSTDiffListWithoutDB() { ); DifferSnapshotInfo snapshotInfo1 = new DifferSnapshotInfo( - "/path/to/dbcp1", UUID.randomUUID(), 3008L, null, null); + "/path/to/dbcp1", UUID.randomUUID(), 3008L, null, Mockito.mock(ManagedRocksDB.class)); DifferSnapshotInfo snapshotInfo2 = new DifferSnapshotInfo( - "/path/to/dbcp2", UUID.randomUUID(), 14980L, null, null); + "/path/to/dbcp2", UUID.randomUUID(), 14980L, null, Mockito.mock(ManagedRocksDB.class)); DifferSnapshotInfo snapshotInfo3 = new DifferSnapshotInfo( - "/path/to/dbcp3", UUID.randomUUID(), 17975L, null, null); + "/path/to/dbcp3", UUID.randomUUID(), 17975L, null, Mockito.mock(ManagedRocksDB.class)); DifferSnapshotInfo snapshotInfo4 = new DifferSnapshotInfo( - "/path/to/dbcp4", UUID.randomUUID(), 18000L, null, null); + "/path/to/dbcp4", UUID.randomUUID(), 18000L, null, Mockito.mock(ManagedRocksDB.class)); + DifferSnapshotInfo snapshotInfo5 = new DifferSnapshotInfo( + "/path/to/dbcp2", UUID.randomUUID(), 0L, null, Mockito.mock(ManagedRocksDB.class)); + DifferSnapshotInfo snapshotInfo6 = new DifferSnapshotInfo( + "/path/to/dbcp2", UUID.randomUUID(), 100L, null, Mockito.mock(ManagedRocksDB.class)); Set snapshotSstFiles1 = ImmutableSet.of("000059", "000053"); Set snapshotSstFiles2 = ImmutableSet.of("000088", "000059", @@ -340,6 +373,8 @@ private static Stream casesGetSSTDiffListWithoutDB() { snapshotSstFiles3, snapshotSstFiles1, ImmutableSet.of("000059", "000053"), + ImmutableSet.of("000066", "000105", "000080", "000087", "000073", + "000095"), ImmutableSet.of("000066", "000105", "000080", "000087", "000073", "000095"), false), @@ -354,6 +389,7 @@ private static Stream casesGetSSTDiffListWithoutDB() { snapshotSstFiles3, ImmutableSet.of("000088", "000105", "000059", "000053", "000095"), ImmutableSet.of("000108"), + ImmutableSet.of("000108"), false), Arguments.of("Test 3: Compaction log file crafted input: " + "Same SST files found during SST expansion", @@ -365,6 +401,7 @@ private static Stream casesGetSSTDiffListWithoutDB() { snapshotSstFiles1Alt1, ImmutableSet.of("000066", "000059", "000053"), ImmutableSet.of("000080", "000087", "000073", "000095"), + ImmutableSet.of("000080", "000087", "000073", "000095"), false), Arguments.of("Test 4: Compaction log file crafted input: " + "Skipping known processed SST.", @@ -376,6 +413,7 @@ private static Stream casesGetSSTDiffListWithoutDB() { snapshotSstFiles1Alt2, Collections.emptySet(), Collections.emptySet(), + Collections.emptySet(), true), Arguments.of("Test 5: Compaction log file hit snapshot" + " generation early exit condition", @@ -387,6 +425,7 @@ private static Stream casesGetSSTDiffListWithoutDB() { snapshotSstFiles1, ImmutableSet.of("000059", "000053"), ImmutableSet.of("000066", "000080", "000087", "000073", "000062"), + ImmutableSet.of("000066", "000080", "000087", "000073", "000062"), false), Arguments.of("Test 6: Compaction log table regular case. " + "Expands expandable SSTs in the initial diff.", @@ -397,6 +436,8 @@ private static Stream casesGetSSTDiffListWithoutDB() { snapshotSstFiles3, snapshotSstFiles1, ImmutableSet.of("000059", "000053"), + ImmutableSet.of("000066", "000105", "000080", "000087", "000073", + "000095"), ImmutableSet.of("000066", "000105", "000080", "000087", "000073", "000095"), false), @@ -411,6 +452,7 @@ private static Stream casesGetSSTDiffListWithoutDB() { snapshotSstFiles3, ImmutableSet.of("000088", "000105", "000059", "000053", "000095"), ImmutableSet.of("000108"), + ImmutableSet.of("000108"), false), Arguments.of("Test 8: Compaction log table crafted input: " + "Same SST files found during SST expansion", @@ -422,6 +464,7 @@ private static Stream casesGetSSTDiffListWithoutDB() { snapshotSstFiles1Alt1, ImmutableSet.of("000066", "000059", "000053"), ImmutableSet.of("000080", "000087", "000073", "000095"), + ImmutableSet.of("000080", "000087", "000073", "000095"), false), Arguments.of("Test 9: Compaction log table crafted input: " + "Skipping known processed SST.", @@ -433,6 +476,7 @@ private static Stream casesGetSSTDiffListWithoutDB() { snapshotSstFiles1Alt2, Collections.emptySet(), Collections.emptySet(), + Collections.emptySet(), true), Arguments.of("Test 10: Compaction log table hit snapshot " + "generation early exit condition", @@ -444,7 +488,33 @@ private static Stream casesGetSSTDiffListWithoutDB() { snapshotSstFiles1, ImmutableSet.of("000059", "000053"), ImmutableSet.of("000066", "000080", "000087", "000073", "000062"), + ImmutableSet.of("000066", "000080", "000087", "000073", "000062"), + false), + Arguments.of("Test 11: Older Compaction log got pruned and source snapshot delta files would be " + + "unreachable", + null, + getPrunedCompactionEntries(false), + snapshotInfo6, + snapshotInfo5, + ImmutableSet.of("10", "11", "8", "9", "12"), + ImmutableSet.of("1", "3", "13", "14"), + ImmutableSet.of("1", "3", "13", "14"), + ImmutableSet.of("2", "8", "9", "12"), + ImmutableSet.of("2", "8", "9", "12"), + false), + Arguments.of("Test 12: Older Compaction log got pruned and source snapshot delta files would be " + + "unreachable", + null, + getPrunedCompactionEntries(true), + snapshotInfo6, + snapshotInfo5, + ImmutableSet.of("10", "11", "8", "9", "12"), + ImmutableSet.of("1", "3", "13", "14"), + ImmutableSet.of("3", "13", "14"), + ImmutableSet.of("4", "5", "8", "9", "12"), + null, false) + ); } @@ -464,10 +534,10 @@ public void testGetSSTDiffListWithoutDB(String description, Set destSnapshotSstFiles, Set expectedSameSstFiles, Set expectedDiffSstFiles, - boolean expectingException) { + Set expectedSSTDiffFiles, + boolean expectingException) throws IOException { boolean exceptionThrown = false; - if (compactionLog != null) { // Construct DAG from compaction log input Arrays.stream(compactionLog.split("\n")).forEach( @@ -500,10 +570,41 @@ public void testGetSSTDiffListWithoutDB(String description, } } + if (expectingException && !exceptionThrown) { + fail("Expecting exception but none thrown."); + } + // Check same and different SST files result assertEquals(expectedSameSstFiles, actualSameSstFiles); assertEquals(expectedDiffSstFiles, actualDiffSstFiles); - + try (MockedStatic 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); + mockedHandler.when(() -> RdbUtil.getLiveSSTFilesForCFs(any(), any())) + .thenAnswer(i -> { + Set sstFiles = i.getArgument(0).equals(srcSnapshot.getRocksDB()) ? srcSnapshotSstFiles + : destSnapshotSstFiles; + return sstFiles.stream().map(fileName -> { + LiveFileMetaData liveFileMetaData = Mockito.mock(LiveFileMetaData.class); + Mockito.when(liveFileMetaData.fileName()).thenReturn("/" + fileName + SST_FILE_EXTENSION); + return liveFileMetaData; + }).collect(Collectors.toList()); + }); + try { + Assertions.assertEquals(Optional.ofNullable(expectedSSTDiffFiles) + .map(files -> files.stream().sorted().collect(Collectors.toList())).orElse(null), + rocksDBCheckpointDiffer.getSSTDiffList(srcSnapshot, destSnapshot) + .map(i -> i.stream().sorted().collect(Collectors.toList())).orElse(null)); + } catch (RuntimeException rtEx) { + if (!expectingException) { + fail("Unexpected exception thrown in test."); + } else { + exceptionThrown = true; + } + } + } if (expectingException && !exceptionThrown) { fail("Expecting exception but none thrown."); } @@ -587,7 +688,7 @@ void diffAllSnapshots(RocksDBCheckpointDiffer differ) int index = 0; for (DifferSnapshotInfo snap : snapshots) { // Returns a list of SST files to be fed into RocksDiff - List sstDiffList = differ.getSSTDiffList(src, snap); + List sstDiffList = differ.getSSTDiffList(src, snap).orElse(Collections.emptyList()); LOG.info("SST diff list from '{}' to '{}': {}", src.getDbPath(), snap.getDbPath(), sstDiffList); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java index c566cae414f..ecb493ecf8f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestOMSnapshotDAG.java @@ -56,6 +56,7 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.time.Duration; +import java.util.Collections; import java.util.List; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; @@ -231,7 +232,7 @@ public void testDAGReconstruction() final File checkpointSnap2 = new File(snap2.getDbPath()); GenericTestUtils.waitFor(checkpointSnap2::exists, 2000, 20000); - List sstDiffList21 = differ.getSSTDiffList(snap2, snap1); + List sstDiffList21 = differ.getSSTDiffList(snap2, snap1).orElse(Collections.emptyList()); LOG.debug("Got diff list: {}", sstDiffList21); // Delete 1000 keys, take a 3rd snapshot, and do another diff @@ -250,13 +251,13 @@ public void testDAGReconstruction() final File checkpointSnap3 = new File(snap3.getDbPath()); GenericTestUtils.waitFor(checkpointSnap3::exists, 2000, 20000); - List sstDiffList32 = differ.getSSTDiffList(snap3, snap2); + List sstDiffList32 = differ.getSSTDiffList(snap3, snap2).orElse(Collections.emptyList()); // snap3-snap1 diff result is a combination of snap3-snap2 and snap2-snap1 - List sstDiffList31 = differ.getSSTDiffList(snap3, snap1); + List sstDiffList31 = differ.getSSTDiffList(snap3, snap1).orElse(Collections.emptyList()); // Same snapshot. Result should be empty list - List sstDiffList22 = differ.getSSTDiffList(snap2, snap2); + List sstDiffList22 = differ.getSSTDiffList(snap2, snap2).orElse(Collections.emptyList()); assertThat(sstDiffList22).isEmpty(); snapDB1.close(); snapDB2.close(); @@ -282,13 +283,13 @@ public void testDAGReconstruction() volumeName, bucketName, "snap3", ((RDBStore) snapDB3.get() .getMetadataManager().getStore()).getDb().getManagedRocksDb()); - List sstDiffList21Run2 = differ.getSSTDiffList(snap2, snap1); + List sstDiffList21Run2 = differ.getSSTDiffList(snap2, snap1).orElse(Collections.emptyList()); assertEquals(sstDiffList21, sstDiffList21Run2); - List sstDiffList32Run2 = differ.getSSTDiffList(snap3, snap2); + List sstDiffList32Run2 = differ.getSSTDiffList(snap3, snap2).orElse(Collections.emptyList()); assertEquals(sstDiffList32, sstDiffList32Run2); - List sstDiffList31Run2 = differ.getSSTDiffList(snap3, snap1); + List sstDiffList31Run2 = differ.getSSTDiffList(snap3, snap1).orElse(Collections.emptyList()); assertEquals(sstDiffList31, sstDiffList31Run2); snapDB1.close(); snapDB2.close(); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java index 6393f12066c..b3d47fece9d 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java @@ -1125,7 +1125,7 @@ Set getDeltaFiles(OmSnapshot fromSnapshot, String diffDir) throws IOException { // TODO: [SNAPSHOT] Refactor the parameter list - final Set deltaFiles = new HashSet<>(); + Optional> deltaFiles = Optional.empty(); // Check if compaction DAG is available, use that if so if (differ != null && fsInfo != null && tsInfo != null && !useFullDiff) { @@ -1139,16 +1139,14 @@ Set getDeltaFiles(OmSnapshot fromSnapshot, LOG.debug("Calling RocksDBCheckpointDiffer"); try { - List sstDiffList = differ.getSSTDiffListWithFullPath(toDSI, - fromDSI, diffDir); - deltaFiles.addAll(sstDiffList); + deltaFiles = differ.getSSTDiffListWithFullPath(toDSI, fromDSI, diffDir).map(HashSet::new); } catch (Exception exception) { LOG.warn("Failed to get SST diff file using RocksDBCheckpointDiffer. " + "It will fallback to full diff now.", exception); } } - if (useFullDiff || deltaFiles.isEmpty()) { + if (useFullDiff || !deltaFiles.isPresent()) { // If compaction DAG is not available (already cleaned up), fall back to // the slower approach. if (!useFullDiff) { @@ -1166,13 +1164,16 @@ Set getDeltaFiles(OmSnapshot fromSnapshot, ((RDBStore)toSnapshot.getMetadataManager().getStore()).getDb() .getManagedRocksDb(), tablesToLookUp); - - deltaFiles.addAll(fromSnapshotFiles); - deltaFiles.addAll(toSnapshotFiles); - RocksDiffUtils.filterRelevantSstFiles(deltaFiles, tablePrefixes); + Set diffFiles = new HashSet<>(); + diffFiles.addAll(fromSnapshotFiles); + diffFiles.addAll(toSnapshotFiles); + RocksDiffUtils.filterRelevantSstFiles(diffFiles, tablePrefixes); + deltaFiles = Optional.of(diffFiles); } - return deltaFiles; + return deltaFiles.orElseThrow(() -> + new IOException("Error getting diff files b/w " + fromSnapshot.getSnapshotTableKey() + " and " + + toSnapshot.getSnapshotTableKey())); } private void validateEstimatedKeyChangesAreInLimits( diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java index 0f2ab615066..75a635cc887 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java @@ -82,6 +82,7 @@ import org.mockito.Mock; import org.mockito.MockedConstruction; import org.mockito.MockedStatic; +import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; @@ -411,7 +412,7 @@ private SnapshotInfo getMockedSnapshotInfo(UUID snapshotId) { } @ParameterizedTest - @ValueSource(ints = {1, 2, 5, 10, 100, 1000, 10000}) + @ValueSource(ints = {0, 1, 2, 5, 10, 100, 1000, 10000}) public void testGetDeltaFilesWithDag(int numberOfFiles) throws IOException { UUID snap1 = UUID.randomUUID(); UUID snap2 = UUID.randomUUID(); @@ -429,7 +430,7 @@ public void testGetDeltaFilesWithDag(int numberOfFiles) throws IOException { any(DifferSnapshotInfo.class), any(DifferSnapshotInfo.class), eq(diffDir)) - ).thenReturn(Lists.newArrayList(randomStrings)); + ).thenReturn(Optional.of(Lists.newArrayList(randomStrings))); ReferenceCounted rcFromSnapshot = omSnapshotManager.getActiveSnapshot(VOLUME_NAME, BUCKET_NAME, snap1.toString()); @@ -441,14 +442,20 @@ public void testGetDeltaFilesWithDag(int numberOfFiles) throws IOException { SnapshotInfo fromSnapshotInfo = getMockedSnapshotInfo(snap1); SnapshotInfo toSnapshotInfo = getMockedSnapshotInfo(snap2); when(jobTableIterator.isValid()).thenReturn(false); - Set deltaFiles = snapshotDiffManager.getDeltaFiles( - fromSnapshot, - toSnapshot, - Arrays.asList("cf1", "cf2"), fromSnapshotInfo, - toSnapshotInfo, false, - Collections.emptyMap(), diffDir); - assertEquals(randomStrings, deltaFiles); - + try (MockedStatic mockedRdbUtil = Mockito.mockStatic(RdbUtil.class, Mockito.CALLS_REAL_METHODS); + MockedStatic mockedRocksDiffUtils = Mockito.mockStatic(RocksDiffUtils.class, + Mockito.CALLS_REAL_METHODS)) { + mockedRdbUtil.when(() -> RdbUtil.getSSTFilesForComparison(any(), any())) + .thenReturn(Collections.singleton(RandomStringUtils.randomAlphabetic(10))); + mockedRocksDiffUtils.when(() -> RocksDiffUtils.filterRelevantSstFiles(any(), any())).thenAnswer(i -> null); + Set deltaFiles = snapshotDiffManager.getDeltaFiles( + fromSnapshot, + toSnapshot, + Arrays.asList("cf1", "cf2"), fromSnapshotInfo, + toSnapshotInfo, false, + Collections.emptyMap(), diffDir); + assertEquals(randomStrings, deltaFiles); + } rcFromSnapshot.close(); rcToSnapshot.close(); } @@ -497,7 +504,7 @@ public void testGetDeltaFilesWithFullDiff(int numberOfFiles, any(DifferSnapshotInfo.class), any(DifferSnapshotInfo.class), anyString())) - .thenReturn(Collections.emptyList()); + .thenReturn(Optional.ofNullable(Collections.emptyList())); } ReferenceCounted rcFromSnapshot =