Skip to content

Commit

Permalink
HDDS-11914. Add test case
Browse files Browse the repository at this point in the history
Change-Id: Iaf7b1bf0b1c986a8956e3fdfa0598dff4f56766f
  • Loading branch information
swamirishi committed Dec 13, 2024
1 parent 2cfd895 commit 8548a53
Showing 1 changed file with 119 additions and 78 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import static java.util.Arrays.asList;
import static java.util.concurrent.TimeUnit.MINUTES;

import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.graph.GraphBuilder;
import java.io.File;
Expand Down Expand Up @@ -101,12 +102,14 @@
import static org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.DEBUG_READ_ALL_DB_KEYS;
import static org.apache.ozone.rocksdiff.RocksDBCheckpointDiffer.SST_FILE_EXTENSION;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.in;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
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.ArgumentMatchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

Expand Down Expand Up @@ -237,26 +240,26 @@ public void cleanUp() {
}
}

private static List<CompactionLogEntry> getPrunedCompactionEntries(boolean prune) {
private static List<CompactionLogEntry> getPrunedCompactionEntries(boolean prune, Map<String, String[]> metadata) {
List<CompactionLogEntry> entries = new ArrayList<>();
if (!prune) {
entries.add(createCompactionEntry(1,
now(),
Arrays.asList("1", "2"),
Arrays.asList("4", "5")));
Arrays.asList("4", "5"), metadata));
}
entries.addAll(Arrays.asList(createCompactionEntry(2,
now(),
Arrays.asList("4", "5"),
Collections.singletonList("10")),
Collections.singletonList("10"), metadata),
createCompactionEntry(3,
now(),
Arrays.asList("3", "13", "14"),
Arrays.asList("6", "7")),
Arrays.asList("6", "7"), metadata),
createCompactionEntry(4,
now(),
Arrays.asList("6", "7"),
Collections.singletonList("11"))));
Collections.singletonList("11"), metadata)));
return entries;
}

Expand Down Expand Up @@ -342,10 +345,12 @@ private static Stream<Arguments> casesGetSSTDiffListWithoutDB() {
"/path/to/dbcp3", UUID.randomUUID(), 17975L, null, Mockito.mock(ManagedRocksDB.class));
DifferSnapshotInfo snapshotInfo4 = new DifferSnapshotInfo(
"/path/to/dbcp4", UUID.randomUUID(), 18000L, null, Mockito.mock(ManagedRocksDB.class));

Map<String, String> prefixMap = ImmutableMap.of("col1", "c", "col2", "d");
DifferSnapshotInfo snapshotInfo5 = new DifferSnapshotInfo(
"/path/to/dbcp2", UUID.randomUUID(), 0L, null, Mockito.mock(ManagedRocksDB.class));
"/path/to/dbcp2", UUID.randomUUID(), 0L, prefixMap, Mockito.mock(ManagedRocksDB.class));
DifferSnapshotInfo snapshotInfo6 = new DifferSnapshotInfo(
"/path/to/dbcp2", UUID.randomUUID(), 100L, null, Mockito.mock(ManagedRocksDB.class));
"/path/to/dbcp2", UUID.randomUUID(), 100L, prefixMap, Mockito.mock(ManagedRocksDB.class));

Set<String> snapshotSstFiles1 = ImmutableSet.of("000059", "000053");
Set<String> snapshotSstFiles2 = ImmutableSet.of("000088", "000059",
Expand Down Expand Up @@ -493,7 +498,7 @@ private static Stream<Arguments> casesGetSSTDiffListWithoutDB() {
Arguments.of("Test 11: Older Compaction log got pruned and source snapshot delta files would be " +
"unreachable",
null,
getPrunedCompactionEntries(false),
getPrunedCompactionEntries(false, Collections.emptyMap()),
snapshotInfo6,
snapshotInfo5,
ImmutableSet.of("10", "11", "8", "9", "12"),
Expand All @@ -505,14 +510,36 @@ private static Stream<Arguments> casesGetSSTDiffListWithoutDB() {
Arguments.of("Test 12: Older Compaction log got pruned and source snapshot delta files would be " +
"unreachable",
null,
getPrunedCompactionEntries(true),
getPrunedCompactionEntries(true, Collections.emptyMap()),
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),
Arguments.of("Test 13: Compaction log to test filtering logic based on range and column family",
null,
getPrunedCompactionEntries(false,
ImmutableMap.of(
"1", new String[]{"a", "c", "col1"},
"3", new String[]{"a", "d", "col2"},
"13", new String[]{"a", "c", "col13"},
"14", new String[]{"a", "c", "col1"},
"2", new String[]{"a", "c", "col1"},
"4", new String[]{"a", "b", "col1"},
"5", new String[]{"b", "b", "col1"},
"10", new String[]{"a", "b", "col1"},
"8", new String[]{"a", "b", "col1"}
)),
snapshotInfo6,
snapshotInfo5,
ImmutableSet.of("10", "11", "8", "9", "12"),
ImmutableSet.of("1", "3", "13", "14"),
ImmutableSet.of("1", "3", "14"),
ImmutableSet.of("2", "8", "9", "12"),
ImmutableSet.of("2", "9", "12"),
false)

);
Expand All @@ -535,78 +562,81 @@ public void testGetSSTDiffListWithoutDB(String description,
Set<String> expectedSameSstFiles,
Set<String> expectedDiffSstFiles,
Set<String> expectedSSTDiffFiles,
boolean expectingException) throws IOException {

boolean exceptionThrown = false;
if (compactionLog != null) {
// Construct DAG from compaction log input
Arrays.stream(compactionLog.split("\n")).forEach(
rocksDBCheckpointDiffer::processCompactionLogLine);
} else if (compactionLogEntries != null) {
compactionLogEntries.forEach(entry ->
rocksDBCheckpointDiffer.addToCompactionLogTable(entry));
} else {
throw new IllegalArgumentException("One of compactionLog and " +
"compactionLogEntries should be non-null.");
}
rocksDBCheckpointDiffer.loadAllCompactionLogs();

Set<String> actualSameSstFiles = new HashSet<>();
Set<String> actualDiffSstFiles = new HashSet<>();

try {
rocksDBCheckpointDiffer.internalGetSSTDiffList(
srcSnapshot,
destSnapshot,
srcSnapshotSstFiles,
destSnapshotSstFiles,
actualSameSstFiles,
actualDiffSstFiles);
} catch (RuntimeException rtEx) {
if (!expectingException) {
fail("Unexpected exception thrown in test.");
boolean expectingException) {
try (MockedStatic<RocksDiffUtils> mockedRocksdiffUtil = Mockito.mockStatic(RocksDiffUtils.class,
Mockito.CALLS_REAL_METHODS)) {
mockedRocksdiffUtil.when(() -> RocksDiffUtils.constructBucketKey(anyString())).thenAnswer(i -> i.getArgument(0));
boolean exceptionThrown = false;
if (compactionLog != null) {
// Construct DAG from compaction log input
Arrays.stream(compactionLog.split("\n")).forEach(
rocksDBCheckpointDiffer::processCompactionLogLine);
} else if (compactionLogEntries != null) {
compactionLogEntries.forEach(entry ->
rocksDBCheckpointDiffer.addToCompactionLogTable(entry));
} else {
exceptionThrown = true;
throw new IllegalArgumentException("One of compactionLog and " +
"compactionLogEntries should be non-null.");
}
}
rocksDBCheckpointDiffer.loadAllCompactionLogs();

if (expectingException && !exceptionThrown) {
fail("Expecting exception but none thrown.");
}
Set<String> actualSameSstFiles = new HashSet<>();
Set<String> actualDiffSstFiles = new HashSet<>();

// 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);
mockedHandler.when(() -> RdbUtil.getLiveSSTFilesForCFs(any(), any()))
.thenAnswer(i -> {
Set<String> 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));
rocksDBCheckpointDiffer.internalGetSSTDiffList(
srcSnapshot,
destSnapshot,
srcSnapshotSstFiles,
destSnapshotSstFiles,
actualSameSstFiles,
actualDiffSstFiles);
} catch (RuntimeException rtEx) {
if (!expectingException) {
fail("Unexpected exception thrown in test.");
} else {
exceptionThrown = true;
}
}
}
if (expectingException && !exceptionThrown) {
fail("Expecting exception but none thrown.");

if (expectingException && !exceptionThrown) {
fail("Expecting exception but none thrown.");
}

// 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);
mockedHandler.when(() -> RdbUtil.getLiveSSTFilesForCFs(any(), any()))
.thenAnswer(i -> {
Set<String> 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.");
}
}
}

Expand Down Expand Up @@ -1558,19 +1588,30 @@ private static Stream<Arguments> sstFilePruningScenarios() {
);
}

private static CompactionLogEntry createCompactionEntry(
long dbSequenceNumber,
long compactionTime,
List<String> inputFiles,
List<String> outputFiles
) {
private static CompactionLogEntry createCompactionEntry(long dbSequenceNumber,
long compactionTime,
List<String> inputFiles,
List<String> outputFiles) {
return createCompactionEntry(dbSequenceNumber, compactionTime, inputFiles, outputFiles, Collections.emptyMap());
}

private static CompactionLogEntry createCompactionEntry(long dbSequenceNumber,
long compactionTime,
List<String> inputFiles,
List<String> outputFiles,
Map<String, String[]> metadata) {
return new CompactionLogEntry.Builder(dbSequenceNumber, compactionTime,
toFileInfoList(inputFiles), toFileInfoList(outputFiles)).build();
toFileInfoList(inputFiles,metadata), toFileInfoList(outputFiles, metadata)).build();
}

private static List<CompactionFileInfo> toFileInfoList(List<String> files) {
private static List<CompactionFileInfo> toFileInfoList(List<String> files,
Map<String, String[]> metadata) {
return files.stream()
.map(fileName -> new CompactionFileInfo.Builder(fileName).build())
.map(fileName -> new CompactionFileInfo.Builder(fileName)
.setStartRange(Optional.ofNullable(metadata.get(fileName)).map(meta -> meta[0]).orElse(null))
.setEndRange(Optional.ofNullable(metadata.get(fileName)).map(meta -> meta[1]).orElse(null))
.setColumnFamily(Optional.ofNullable(metadata.get(fileName)).map(meta -> meta[2]).orElse(null))
.build())
.collect(Collectors.toList());
}

Expand Down

0 comments on commit 8548a53

Please sign in to comment.