Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_CHECKPOINT_DIR;
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_INCLUDE_SNAPSHOT_DATA;
import static org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_REQUEST_FLUSH;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_KEY;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
Expand Down Expand Up @@ -143,11 +144,11 @@ void shutdown() {

private void setupCluster() throws Exception {
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
conf.setBoolean(OZONE_ACL_ENABLED, false);
conf.set(OZONE_ADMINISTRATORS, OZONE_ADMINISTRATORS_WILDCARD);
cluster.waitForClusterToBeReady();
client = cluster.newClient();
om = cluster.getOzoneManager();
conf.setBoolean(OZONE_ACL_ENABLED, false);
conf.set(OZONE_ADMINISTRATORS, OZONE_ADMINISTRATORS_WILDCARD);
}

private void setupMocks() throws Exception {
Expand Down Expand Up @@ -224,12 +225,44 @@ public void write(int b) throws IOException {
doCallRealMethod().when(omDbCheckpointServletMock).getSstBackupDir();
}

@Test
void testContentsOfTarballWithSnapshot() throws Exception {
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testTarballBatching(boolean includeSnapshots) throws Exception {
String volumeName = "vol" + RandomStringUtils.secure().nextNumeric(5);
String bucketName = "buck" + RandomStringUtils.secure().nextNumeric(5);
AtomicReference<DBCheckpoint> realCheckpoint = new AtomicReference<>();
setupClusterAndMocks(volumeName, bucketName, realCheckpoint);
setupClusterAndMocks(volumeName, bucketName, realCheckpoint, includeSnapshots);
long maxFileSizeLimit = 4096;
om.getConfiguration().setLong(OZONE_OM_RATIS_SNAPSHOT_MAX_TOTAL_SST_SIZE_KEY, maxFileSizeLimit);
// Get the tarball.
omDbCheckpointServletMock.doGet(requestMock, responseMock);
String testDirName = folder.resolve("testDir").toString();
String newDbDirName = testDirName + OM_KEY_PREFIX + OM_DB_NAME;
File newDbDir = new File(newDbDirName);
assertTrue(newDbDir.mkdirs());
FileUtil.unTar(tempFile, newDbDir);
long totalSize;
try (Stream<Path> list = Files.list(newDbDir.toPath())) {
totalSize = list.mapToLong(path -> path.toFile().length()).sum();
}
boolean obtainedFilesUnderMaxLimit = totalSize < maxFileSizeLimit;
if (!includeSnapshots) {
// If includeSnapshotData flag is set to false , it always sends all data
// in one batch and doesn't respect the max size config. This is how Recon
// uses it today.
assertFalse(obtainedFilesUnderMaxLimit);
} else {
assertTrue(obtainedFilesUnderMaxLimit);
}
Comment on lines +248 to +256
Copy link
Contributor

@jojochuang jojochuang Jul 31, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can simplify as:

Suggested change
boolean obtainedFilesUnderMaxLimit = totalSize < maxFileSizeLimit;
if (!includeSnapshots) {
// If includeSnapshotData flag is set to false , it always sends all data
// in one batch and doesn't respect the max size config. This is how Recon
// uses it today.
assertFalse(obtainedFilesUnderMaxLimit);
} else {
assertTrue(obtainedFilesUnderMaxLimit);
}
// If includeSnapshotData flag is set to false , it always sends all data
// in one batch and doesn't respect the max size config. This is how Recon
// uses it today.
assertEquals(includeSnapshots, totalSize < maxFileSizeLimit);

}

@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testContentsOfTarballWithSnapshot(boolean includeSnapshot) throws Exception {
String volumeName = "vol" + RandomStringUtils.secure().nextNumeric(5);
String bucketName = "buck" + RandomStringUtils.secure().nextNumeric(5);
AtomicReference<DBCheckpoint> realCheckpoint = new AtomicReference<>();
setupClusterAndMocks(volumeName, bucketName, realCheckpoint, includeSnapshot);
DBStore dbStore = om.getMetadataManager().getStore();
// Get the tarball.
omDbCheckpointServletMock.doGet(requestMock, responseMock);
Expand Down Expand Up @@ -258,9 +291,11 @@ void testContentsOfTarballWithSnapshot() throws Exception {
Path checkpointLocation = realCheckpoint.get().getCheckpointLocation();
populateInodesOfFilesInDirectory(dbStore, checkpointLocation,
inodesFromOmDataDir, hardLinkMapFromOmData);
for (String snapshotPath : snapshotPaths) {
populateInodesOfFilesInDirectory(dbStore, Paths.get(snapshotPath),
inodesFromOmDataDir, hardLinkMapFromOmData);
if (includeSnapshot) {
for (String snapshotPath : snapshotPaths) {
populateInodesOfFilesInDirectory(dbStore, Paths.get(snapshotPath),
inodesFromOmDataDir, hardLinkMapFromOmData);
}
}
populateInodesOfFilesInDirectory(dbStore, Paths.get(dbStore.getRocksDBCheckpointDiffer().getSSTBackupDir()),
inodesFromOmDataDir, hardLinkMapFromOmData);
Expand Down Expand Up @@ -296,7 +331,7 @@ public void testSnapshotDBConsistency() throws Exception {
String volumeName = "vol" + RandomStringUtils.secure().nextNumeric(5);
String bucketName = "buck" + RandomStringUtils.secure().nextNumeric(5);
AtomicReference<DBCheckpoint> realCheckpoint = new AtomicReference<>();
setupClusterAndMocks(volumeName, bucketName, realCheckpoint);
setupClusterAndMocks(volumeName, bucketName, realCheckpoint, true);
List<OzoneSnapshot> snapshots = new ArrayList<>();
client.getObjectStore().listSnapshot(volumeName, bucketName, "", null)
.forEachRemaining(snapshots::add);
Expand Down Expand Up @@ -420,11 +455,12 @@ private void writeDummyKeyToDeleteTableOfSnapshotDB(OzoneSnapshot snapshotToModi
}

private void setupClusterAndMocks(String volumeName, String bucketName,
AtomicReference<DBCheckpoint> realCheckpoint) throws Exception {
AtomicReference<DBCheckpoint> realCheckpoint, boolean includeSnapshots) throws Exception {
setupCluster();
setupMocks();
om.getKeyManager().getSnapshotSstFilteringService().pause();
when(requestMock.getParameter(OZONE_DB_CHECKPOINT_INCLUDE_SNAPSHOT_DATA)).thenReturn("true");
when(requestMock.getParameter(OZONE_DB_CHECKPOINT_INCLUDE_SNAPSHOT_DATA))
.thenReturn(String.valueOf(includeSnapshots));
// Create a "spy" dbstore keep track of the checkpoint.
writeData(volumeName, bucketName, true);
DBStore dbStore = om.getMetadataManager().getStore();
Expand Down