This is an automated email from the ASF dual-hosted git repository.

aswinshakil pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 96cc2236109 HDDS-14652. Handle NoSuchFileException during the 
bootstrap tarball transfer. (#9784)
96cc2236109 is described below

commit 96cc223610927a94ba21ca9dc165013116f05c23
Author: Sadanand Shenoy <[email protected]>
AuthorDate: Fri Feb 27 04:26:54 2026 +0530

    HDDS-14652. Handle NoSuchFileException during the bootstrap tarball 
transfer. (#9784)
    
    Co-authored-by: Sadanand Shenoy <[email protected]>
---
 .../TestOMDbCheckpointServletInodeBasedXfer.java   | 48 +++++++++++++++++++
 .../org/apache/hadoop/ozone/om/OMDBArchiver.java   |  4 ++
 .../om/OMDBCheckpointServletInodeBasedXfer.java    | 55 +++++++++++++++++-----
 3 files changed, 96 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 8787369498c..6f210eb9bc9 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
@@ -33,6 +33,7 @@
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
+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.anyCollection;
@@ -57,6 +58,7 @@
 import java.io.OutputStream;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
@@ -236,6 +238,8 @@ public void write(int b) throws IOException {
         .writeDbDataToStream(any(), any(), any(), any(), any());
     doCallRealMethod().when(omDbCheckpointServletMock)
         .collectFilesFromDir(any(), any(), any(), anyBoolean(), any());
+    doCallRealMethod().when(omDbCheckpointServletMock)
+        .collectFilesFromDir(any(), any(), any(), anyBoolean(), any(), 
anyBoolean());
     
doCallRealMethod().when(omDbCheckpointServletMock).collectDbDataToTransfer(any(),
 any(), any());
 
     when(omDbCheckpointServletMock.getBootstrapStateLock())
@@ -852,6 +856,50 @@ public void 
testCheckpointIncludesSnapshotsFromFrozenState() throws Exception {
     }
   }
 
+
+  /**
+   * Verifies that when a file is deleted before/during transfer, the servlet
+   * gracefully skips it without failing (handles NoSuchFileException).
+   */
+  @Test
+  public void testCollectFilesFromDirSkipsDeletedFile() throws Exception {
+    OMDBCheckpointServletInodeBasedXfer servlet = new 
OMDBCheckpointServletInodeBasedXfer();
+    Path dbDir = Files.createTempDirectory(folder, "dbdir-");
+    Path sstFile1 = dbDir.resolve("file1.sst");
+    Path sstFile2 = dbDir.resolve("file2.sst");
+    Path sstFile3 = dbDir.resolve("file3.sst");
+    Files.write(sstFile1, "content1".getBytes(StandardCharsets.UTF_8));
+    Files.write(sstFile2, "content2".getBytes(StandardCharsets.UTF_8));
+    Files.write(sstFile3, "content3".getBytes(StandardCharsets.UTF_8));
+    List<Path> filesUnderDir = new ArrayList<>();
+    filesUnderDir.add(sstFile1);
+    filesUnderDir.add(sstFile2);
+    filesUnderDir.add(sstFile3);
+    // Delete file2 before transfer , same as pruner
+    Files.delete(sstFile2);
+    OMDBArchiver archiver = new OMDBArchiver();
+    Path tmpDir = folder.resolve("tmp-deleted-file-test");
+    Files.createDirectories(tmpDir);
+    archiver.setTmpDir(tmpDir);
+    OMDBArchiver archiverSpy = spy(archiver);
+
+    Set<String> sstFilesToExclude = new HashSet<>();
+    AtomicLong maxTotalSstSize = new AtomicLong(Long.MAX_VALUE);
+
+    doAnswer(invocation -> archiver.recordFileEntry(
+        invocation.getArgument(0), invocation.getArgument(1)))
+        .when(archiverSpy).recordFileEntry(any(), anyString());
+    boolean result = servlet.collectFilesFromDir(sstFilesToExclude, 
filesUnderDir.stream(),
+        maxTotalSstSize, true, archiverSpy, true);
+    assertTrue(result);
+    verify(archiverSpy, times(2)).recordFileEntry(any(), anyString());
+    // should throw exception when flag is set to false.
+    assertThrows(NoSuchFileException.class, () -> {
+      servlet.collectFilesFromDir(sstFilesToExclude, filesUnderDir.stream(),
+          maxTotalSstSize, true, archiverSpy, false);
+    });
+  }
+
   private void writeDummyKeyToDeleteTableOfSnapshotDB(OzoneSnapshot 
snapshotToModify, String bucketName,
       String volumeName, String keyName)
       throws IOException {
diff --git 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBArchiver.java
 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBArchiver.java
index 9a2ec9ec87d..a1c63db16a4 100644
--- 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBArchiver.java
+++ 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBArchiver.java
@@ -74,6 +74,10 @@ public void recordHardLinkMapping(String absolutePath, 
String fileId) {
     hardLinkFileMap.put(absolutePath, fileId);
   }
 
+  public void removeHardLinkMapping(String absolutePath) {
+    hardLinkFileMap.remove(absolutePath);
+  }
+
   public boolean isCompleted() {
     return completed;
   }
diff --git 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServletInodeBasedXfer.java
 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServletInodeBasedXfer.java
index 817dfc30b44..5e305204328 100644
--- 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServletInodeBasedXfer.java
+++ 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMDBCheckpointServletInodeBasedXfer.java
@@ -35,6 +35,7 @@
 import java.io.OutputStream;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.nio.file.StandardOpenOption;
@@ -304,7 +305,9 @@ public void collectDbDataToTransfer(HttpServletRequest 
request,
             }
             collectFilesFromDir(sstFilesToExclude, getCompactionLogDir(), 
maxTotalSstSize, false, omdbArchiver);
             try (Stream<Path> backupFiles = sstBackupFiles.stream()) {
-              collectFilesFromDir(sstFilesToExclude, backupFiles, 
maxTotalSstSize, false, omdbArchiver);
+              // since pruner can delete the files after it has been loaded 
into sstBackupFiles,
+              // NoSuchFileException needs to be caught
+              collectFilesFromDir(sstFilesToExclude, backupFiles, 
maxTotalSstSize, false, omdbArchiver, true);
             }
             Collection<Path> snapshotLocalPropertyFiles = 
getSnapshotLocalDataPaths(localDataManager,
                 snapshotInCheckpoint.keySet());
@@ -478,6 +481,12 @@ boolean collectFilesFromDir(Set<String> sstFilesToExclude, 
Path dbDir, AtomicLon
     }
   }
 
+  private boolean collectFilesFromDir(Set<String> sstFilesToExclude, 
Stream<Path> files,
+      AtomicLong maxTotalSstSize, boolean onlySstFile, OMDBArchiver 
omdbArchiver) throws IOException {
+    return collectFilesFromDir(sstFilesToExclude, files, maxTotalSstSize,
+        onlySstFile, omdbArchiver, false);
+  }
+
   /**
    * Collects database files to the archive, handling deduplication based on 
inode IDs.
    * Here the dbDir could either be a snapshot db directory, the active om.db,
@@ -495,8 +504,9 @@ boolean collectFilesFromDir(Set<String> sstFilesToExclude, 
Path dbDir, AtomicLon
    * @return true if processing should continue, false if size limit reached
    * @throws IOException if an I/O error occurs
    */
-  private boolean collectFilesFromDir(Set<String> sstFilesToExclude, 
Stream<Path> files, AtomicLong maxTotalSstSize,
-      boolean onlySstFile, OMDBArchiver omdbArchiver) throws IOException {
+  boolean collectFilesFromDir(Set<String> sstFilesToExclude, Stream<Path> 
files,
+      AtomicLong maxTotalSstSize, boolean onlySstFile, OMDBArchiver 
omdbArchiver,
+      boolean ignoreNoSuchFileException) throws IOException {
     long bytesRecorded = 0L;
     int filesWritten = 0;
     Iterable<Path> iterable = files::iterator;
@@ -505,7 +515,17 @@ private boolean collectFilesFromDir(Set<String> 
sstFilesToExclude, Stream<Path>
         if (onlySstFile && !dbFile.toString().endsWith(ROCKSDB_SST_SUFFIX)) {
           continue;
         }
-        String fileId = 
OmSnapshotUtils.getFileInodeAndLastModifiedTimeString(dbFile);
+        String fileId;
+        try {
+          fileId = 
OmSnapshotUtils.getFileInodeAndLastModifiedTimeString(dbFile);
+        } catch (NoSuchFileException nsfe) {
+          if (ignoreNoSuchFileException) {
+            LOG.warn("File {} not found.", dbFile);
+            logFileNoLongerExists(dbFile);
+            continue;
+          }
+          throw nsfe;
+        }
         String path = dbFile.toFile().getAbsolutePath();
         // if the file is in the om checkpoint dir, then we need to change the 
path to point to the OM DB.
         if (path.contains(OM_CHECKPOINT_DIR)) {
@@ -513,14 +533,23 @@ private boolean collectFilesFromDir(Set<String> 
sstFilesToExclude, Stream<Path>
         }
         omdbArchiver.recordHardLinkMapping(path, fileId);
         if (!sstFilesToExclude.contains(fileId)) {
-          long fileSize = Files.size(dbFile);
-          if (maxTotalSstSize.get() - fileSize <= 0) {
-            return false;
+          try {
+            long fileSize = Files.size(dbFile);
+            if (maxTotalSstSize.get() - fileSize <= 0) {
+              return false;
+            }
+            bytesRecorded += omdbArchiver.recordFileEntry(dbFile.toFile(), 
fileId);
+            filesWritten++;
+            maxTotalSstSize.addAndGet(-fileSize);
+            sstFilesToExclude.add(fileId);
+          } catch (NoSuchFileException e) {
+            if (ignoreNoSuchFileException) {
+              logFileNoLongerExists(dbFile);
+              omdbArchiver.removeHardLinkMapping(path);
+            } else {
+              throw e;
+            }
           }
-          bytesRecorded += omdbArchiver.recordFileEntry(dbFile.toFile(), 
fileId);
-          filesWritten++;
-          maxTotalSstSize.addAndGet(-fileSize);
-          sstFilesToExclude.add(fileId);
         }
       }
     }
@@ -529,6 +558,10 @@ private boolean collectFilesFromDir(Set<String> 
sstFilesToExclude, Stream<Path>
     return true;
   }
 
+  private void logFileNoLongerExists(Path dbFile) {
+    LOG.warn("Not writing DB file : {} to archive as it no longer exists", 
dbFile);
+  }
+
   /**
    * Creates a database checkpoint and copies compaction log and SST backup 
files
    * into the given temporary directory.


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

Reply via email to