From ee370d8a50e8dacf1bad1743f52ee6d4fbbe6122 Mon Sep 17 00:00:00 2001 From: Sadanand Shenoy Date: Thu, 31 Jul 2025 12:58:08 +0530 Subject: [PATCH 1/2] HDDS-13227. Integration test for inode based bootstrap flow --- ...stOMDbCheckpointServletInodeBasedXfer.java | 60 +++++++++++++++---- 1 file changed, 49 insertions(+), 11 deletions(-) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServletInodeBasedXfer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServletInodeBasedXfer.java index 454f7a918111..fbddd9899151 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServletInodeBasedXfer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServletInodeBasedXfer.java @@ -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; @@ -94,6 +95,8 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.rocksdb.ColumnFamilyDescriptor; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.DBOptions; @@ -132,11 +135,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 { @@ -208,12 +211,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 realCheckpoint = new AtomicReference<>(); + 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 = 0; + for (File f : newDbDir.listFiles()) { + totalSize += f.length(); + } + 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); + } + } + + @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 realCheckpoint = new AtomicReference<>(); - setupClusterAndMocks(volumeName, bucketName, realCheckpoint); + setupClusterAndMocks(volumeName, bucketName, realCheckpoint, includeSnapshot); DBStore dbStore = om.getMetadataManager().getStore(); // Get the tarball. omDbCheckpointServletMock.doGet(requestMock, responseMock); @@ -242,9 +277,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); @@ -280,7 +317,7 @@ public void testSnapshotDBConsistency() throws Exception { String volumeName = "vol" + RandomStringUtils.secure().nextNumeric(5); String bucketName = "buck" + RandomStringUtils.secure().nextNumeric(5); AtomicReference realCheckpoint = new AtomicReference<>(); - setupClusterAndMocks(volumeName, bucketName, realCheckpoint); + setupClusterAndMocks(volumeName, bucketName, realCheckpoint, true); List snapshots = new ArrayList<>(); client.getObjectStore().listSnapshot(volumeName, bucketName, "", null) .forEachRemaining(snapshots::add); @@ -349,11 +386,12 @@ private void writeDummyKeyToDeleteTableOfSnapshotDB(OzoneSnapshot snapshotToModi } private void setupClusterAndMocks(String volumeName, String bucketName, - AtomicReference realCheckpoint) throws Exception { + AtomicReference 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(); From 60e4889f63b7b8f780acc0139e03f2f9c29c5352 Mon Sep 17 00:00:00 2001 From: Sadanand Shenoy Date: Thu, 31 Jul 2025 14:52:47 +0530 Subject: [PATCH 2/2] fix findbugs --- .../ozone/om/TestOMDbCheckpointServletInodeBasedXfer.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServletInodeBasedXfer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServletInodeBasedXfer.java index 381a7f92f7e8..a43b0832c960 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServletInodeBasedXfer.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMDbCheckpointServletInodeBasedXfer.java @@ -241,9 +241,9 @@ public void testTarballBatching(boolean includeSnapshots) throws Exception { File newDbDir = new File(newDbDirName); assertTrue(newDbDir.mkdirs()); FileUtil.unTar(tempFile, newDbDir); - long totalSize = 0; - for (File f : newDbDir.listFiles()) { - totalSize += f.length(); + long totalSize; + try (Stream list = Files.list(newDbDir.toPath())) { + totalSize = list.mapToLong(path -> path.toFile().length()).sum(); } boolean obtainedFilesUnderMaxLimit = totalSize < maxFileSizeLimit; if (!includeSnapshots) {