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

weichiu 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 b46e6b26860 HDDS-13150. Fixed SnapshotLimitCheck when failures occur. 
(#8532)
b46e6b26860 is described below

commit b46e6b26860251b755c748ed3ae7b24a77b034cf
Author: SaketaChalamchala <[email protected]>
AuthorDate: Mon Jun 9 19:01:20 2025 -0700

    HDDS-13150. Fixed SnapshotLimitCheck when failures occur. (#8532)
    
    Co-authored-by: saketa <[email protected]>
---
 .../request/snapshot/OMSnapshotCreateRequest.java  |  3 +-
 .../snapshot/TestOMSnapshotCreateRequest.java      | 46 ++++++++++++++++++++++
 2 files changed, 47 insertions(+), 2 deletions(-)

diff --git 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java
 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java
index 57c4bb129bd..66051851a2e 100644
--- 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java
+++ 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java
@@ -202,6 +202,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager 
ozoneManager, Execut
       omClientResponse = new OMSnapshotCreateResponse(
           createErrorOMResponse(omResponse, exception));
     } finally {
+      ozoneManager.getOmSnapshotManager().decrementInFlightSnapshotCount();
       if (acquiredSnapshotLock) {
         mergeOmLockDetails(
             omMetadataManager.getLock().releaseWriteLock(SNAPSHOT_LOCK,
@@ -291,8 +292,6 @@ private void addSnapshotInfoToSnapshotChainAndCache(
         removeSnapshotInfoFromSnapshotChainManager(snapshotChainManager,
             snapshotInfo);
         throw new IOException(exception.getMessage(), exception);
-      } finally {
-        ozoneManager.getOmSnapshotManager().decrementInFlightSnapshotCount();
       }
     }
   }
diff --git 
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotCreateRequest.java
 
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotCreateRequest.java
index a45991816b1..a18ed38e3a4 100644
--- 
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotCreateRequest.java
+++ 
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotCreateRequest.java
@@ -56,6 +56,7 @@
 import 
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
 import 
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
 import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.DisplayName;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.ValueSource;
@@ -318,6 +319,51 @@ public void testSnapshotLimit() throws Exception {
     assertEquals(OMException.ResultCodes.TOO_MANY_SNAPSHOTS, 
omException.getResult());
   }
 
+  @DisplayName("Snapshot limit is enforced even after failed creation 
attempts")
+  @Test
+  public void testSnapshotLimitWithFailures() throws Exception {
+    when(getOzoneManager().isAdmin(any())).thenReturn(true);
+    getOzoneManager().getOmSnapshotManager().close();
+    
getOzoneManager().getConfiguration().setInt(OZONE_OM_FS_SNAPSHOT_MAX_LIMIT, 2);
+    OmSnapshotManager omSnapshotManager = new 
OmSnapshotManager(getOzoneManager());
+    
when(getOzoneManager().getOmSnapshotManager()).thenReturn(omSnapshotManager);
+
+    // Create first snapshot successfully
+    String key1 = getTableKey(getVolumeName(), getBucketName(), snapshotName1);
+    OMRequest omRequest = createSnapshotRequest(getVolumeName(), 
getBucketName(), snapshotName1);
+    OMSnapshotCreateRequest omSnapshotCreateRequest = doPreExecute(omRequest);
+    omSnapshotCreateRequest.validateAndUpdateCache(getOzoneManager(), 1);
+    assertNotNull(getOmMetadataManager().getSnapshotInfoTable().get(key1));
+
+    // Snapshot creation failure
+    OMRequest snapshotRequestFail = createSnapshotRequest(getVolumeName(), 
getBucketName(), snapshotName1);
+    OMSnapshotCreateRequest omSnapshotCreateRequestFail = 
doPreExecute(snapshotRequestFail);
+    OMClientResponse omClientResponse =
+        omSnapshotCreateRequestFail.validateAndUpdateCache(getOzoneManager(), 
2);
+    OMResponse omResponse = omClientResponse.getOMResponse();
+    assertNotNull(omResponse.getCreateSnapshotResponse());
+    assertEquals(OzoneManagerProtocolProtos.Status.FILE_ALREADY_EXISTS, 
omResponse.getStatus());
+
+    // Second snapshot in-flight
+    String key2 = getTableKey(getVolumeName(), getBucketName(), snapshotName2);
+    OMRequest snapshotRequest2 = createSnapshotRequest(getVolumeName(), 
getBucketName(), snapshotName2);
+    OMSnapshotCreateRequest omSnapshotCreateRequest2 = 
doPreExecute(snapshotRequest2);
+
+    // Third snapshot should fail as total 3 > limit 2 (1 in chain + 1 
in-flight + 1 new)
+    OMRequest snapshotRequest3 = createSnapshotRequest(getVolumeName(), 
getBucketName(), snapshotName3);
+    OMException omException = assertThrows(OMException.class, () -> 
doPreExecute(snapshotRequest3));
+    assertEquals(OMException.ResultCodes.TOO_MANY_SNAPSHOTS, 
omException.getResult());
+
+    // Complete the in-flight snapshot successfully
+    omSnapshotCreateRequest2.validateAndUpdateCache(getOzoneManager(), 3);
+    assertNotNull(getOmMetadataManager().getSnapshotInfoTable().get(key2));
+
+    // Another snapshot should fail as total 3 > limit 2 (2 in chain + 1 new)
+    OMRequest snapshotRequest4 = createSnapshotRequest(getVolumeName(), 
getBucketName(), snapshotName4);
+    omException = assertThrows(OMException.class, () -> 
doPreExecute(snapshotRequest4));
+    assertEquals(OMException.ResultCodes.TOO_MANY_SNAPSHOTS, 
omException.getResult());
+  }
+
   private void renameKey(String fromKey, String toKey, long offset)
       throws IOException {
     OmKeyInfo toKeyInfo = addKey(toKey, offset + 1L);


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

Reply via email to