Skip to content
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
Show all changes
22 commits
Select commit Hold shift + click to select a range
a4d878e
HDDS-12596. OM fs snapshot max limit is not enforced
peterxcli Mar 25, 2025
f14a095
fix compile error
peterxcli Mar 25, 2025
fe3780c
Should set default snapshot limit for TestOMSnapshotRequestResponse b…
peterxcli Mar 25, 2025
39d87be
Set default filesystem snapshot max limit in TestOMKeyRequest setup
peterxcli Mar 25, 2025
8338f29
Addressed comment
peterxcli Mar 27, 2025
0834515
Move snapshot validation to preExecute and add new TOO_MANY_SNAPSHOTS…
peterxcli Mar 28, 2025
3c2727d
Use snapshotChain size to validete snapshot limit
peterxcli Mar 30, 2025
7b52f1a
Cleanup test
peterxcli Apr 3, 2025
2607c3e
Count inflight snapshot, too
peterxcli Apr 3, 2025
78bc4c4
fix test
peterxcli Apr 3, 2025
3b367e4
Merge remote-tracking branch 'upstream/master' into hdds12596-om-fs-s…
peterxcli Apr 3, 2025
902b507
Revert "Count inflight snapshot, too"
peterxcli Apr 8, 2025
65922fb
Revert "fix test"
peterxcli Apr 8, 2025
3cb97fe
Rename `validateSnapshotLimit` to `snapshotLimitCheck` in snapshotMan…
peterxcli Apr 8, 2025
0d2d15b
Merge remote-tracking branch 'upstream/master' into hdds12596-om-fs-s…
peterxcli Apr 15, 2025
8f0bab2
Add snaphot inflight count
peterxcli Apr 15, 2025
6465e9f
Reset snapshot inflight count when om become leader
peterxcli Apr 15, 2025
385eeb1
add sync block on inFlightSnapshotCount
peterxcli Apr 25, 2025
ede525e
Merge remote-tracking branch 'upstream/master' into hdds12596-om-fs-s…
peterxcli Apr 25, 2025
55cb418
use AtomicReference for exception handling
peterxcli Apr 25, 2025
1106481
move snapshotLimitCheck to the place that wont have further error
peterxcli Apr 26, 2025
be31e15
(CI affected by HDDS-12173 flakiness)Merge remote-tracking branch 'up…
peterxcli Apr 26, 2025
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 @@ -479,6 +479,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
private final BucketUtilizationMetrics bucketUtilizationMetrics;

private boolean fsSnapshotEnabled;
private int fsSnapshotMaxLimit;

private String omHostName;

Expand Down Expand Up @@ -988,6 +989,9 @@ private void instantiateServices(boolean withNewSnapshot) throws IOException {
fsSnapshotEnabled = configuration.getBoolean(
OMConfigKeys.OZONE_FILESYSTEM_SNAPSHOT_ENABLED_KEY,
OMConfigKeys.OZONE_FILESYSTEM_SNAPSHOT_ENABLED_DEFAULT);
fsSnapshotMaxLimit = configuration.getInt(
OMConfigKeys.OZONE_OM_FS_SNAPSHOT_MAX_LIMIT,
OMConfigKeys.OZONE_OM_FS_SNAPSHOT_MAX_LIMIT_DEFAULT);
omSnapshotManager = new OmSnapshotManager(this);

// Snapshot metrics
Expand Down Expand Up @@ -4365,6 +4369,15 @@ public boolean isFilesystemSnapshotEnabled() {
return fsSnapshotEnabled;
}

/**
* Get the maximum number of Ozone filesystem snapshots allowed.
*
* @return the maximum number of Ozone filesystem snapshots allowed.
*/
public int getFsSnapshotMaxLimit() {
return fsSnapshotMaxLimit;
}

/**
* Get DB updates since a specific sequence number.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -165,6 +165,14 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, Execut
throw new OMException("Snapshot already exists", FILE_ALREADY_EXISTS);
}

// Check snapshot limit
int maxSnapshots = ozoneManager.getFsSnapshotMaxLimit();
if (omMetrics.getNumSnapshotActive() >= maxSnapshots) {
throw new OMException(
String.format("Snapshot limit of %d reached. Cannot create more snapshots.", maxSnapshots),
OMException.ResultCodes.INVALID_SNAPSHOT_ERROR);
}

// Note down RDB latest transaction sequence number, which is used
// as snapshot generation in the Differ.
final long dbLatestSequenceNumber =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,17 @@

package org.apache.hadoop.ozone.om.request.key;

import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_FS_SNAPSHOT_MAX_LIMIT_DEFAULT;
import static org.apache.hadoop.ozone.om.request.OMRequestTestUtils.setupReplicationConfigValidation;
import static org.junit.jupiter.api.Assertions.assertEquals;
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.assertTrue;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.anyInt;
import static org.mockito.Mockito.anyLong;
import static org.mockito.Mockito.anyString;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.framework;
import static org.mockito.Mockito.mock;
Expand Down Expand Up @@ -153,6 +154,7 @@ public void setup() throws Exception {
when(ozoneManager.getDeletionMetrics()).thenReturn(delMetrics);
when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
when(ozoneManager.getConfiguration()).thenReturn(ozoneConfiguration);
when(ozoneManager.getFsSnapshotMaxLimit()).thenReturn(OZONE_OM_FS_SNAPSHOT_MAX_LIMIT_DEFAULT);
OMLayoutVersionManager lvm = mock(OMLayoutVersionManager.class);
when(lvm.isAllowed(anyString())).thenReturn(true);
when(ozoneManager.getVersionManager()).thenReturn(lvm);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
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.mockito.Mockito.any;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.when;

import java.io.IOException;
Expand Down Expand Up @@ -264,6 +264,39 @@ public void testEntryExists() throws Exception {
assertEquals(2, getOmMetrics().getNumSnapshotCreates());
}

@Test
public void testSnapshotLimit() throws Exception {
when(getOzoneManager().isAdmin(any())).thenReturn(true);

when(getOzoneManager().getFsSnapshotMaxLimit()).thenReturn(1);

String key1 = getTableKey(getVolumeName(), getBucketName(), snapshotName1);

OMRequest omRequest =
createSnapshotRequest(getVolumeName(), getBucketName(), snapshotName1);
OMSnapshotCreateRequest omSnapshotCreateRequest = doPreExecute(omRequest);

assertNull(getOmMetadataManager().getSnapshotInfoTable().get(key1));
omSnapshotCreateRequest.validateAndUpdateCache(getOzoneManager(), 1);

assertNotNull(getOmMetadataManager().getSnapshotInfoTable().get(key1));

// Should fail as snapshot limit is 1
omRequest = createSnapshotRequest(getVolumeName(), getBucketName(), snapshotName2);
omSnapshotCreateRequest = doPreExecute(omRequest);
OMClientResponse omClientResponse =
omSnapshotCreateRequest.validateAndUpdateCache(getOzoneManager(), 2);

OMResponse omResponse = omClientResponse.getOMResponse();
assertNotNull(omResponse.getCreateSnapshotResponse());
assertEquals(OzoneManagerProtocolProtos.Status.INVALID_SNAPSHOT_ERROR,
omResponse.getStatus());

assertEquals(1, getOmMetrics().getNumSnapshotCreateFails());
assertEquals(1, getOmMetrics().getNumSnapshotActive());
assertEquals(2, getOmMetrics().getNumSnapshotCreates());
}

private void renameKey(String fromKey, String toKey, long offset)
throws IOException {
OmKeyInfo toKeyInfo = addKey(toKey, offset + 1L);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.hadoop.ozone.om.snapshot;

import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_FS_SNAPSHOT_MAX_LIMIT_DEFAULT;
import static org.apache.hadoop.ozone.om.request.OMRequestTestUtils.createOmKeyInfo;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
Expand Down Expand Up @@ -167,6 +168,7 @@ public void baseSetup() throws Exception {
omMetadataManager);
omSnapshotManager = new OmSnapshotManager(ozoneManager);
when(ozoneManager.getOmSnapshotManager()).thenReturn(omSnapshotManager);
when(ozoneManager.getFsSnapshotMaxLimit()).thenReturn(OZONE_OM_FS_SNAPSHOT_MAX_LIMIT_DEFAULT);
}

@AfterEach
Expand Down
Loading