Skip to content

Commit a4d878e

Browse files
committed
HDDS-12596. OM fs snapshot max limit is not enforced
1 parent a1616ae commit a4d878e

File tree

3 files changed

+57
-1
lines changed

3 files changed

+57
-1
lines changed

hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -479,6 +479,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
479479
private final BucketUtilizationMetrics bucketUtilizationMetrics;
480480

481481
private boolean fsSnapshotEnabled;
482+
private int fsSnapshotMaxLimit;
482483

483484
private String omHostName;
484485

@@ -988,6 +989,9 @@ private void instantiateServices(boolean withNewSnapshot) throws IOException {
988989
fsSnapshotEnabled = configuration.getBoolean(
989990
OMConfigKeys.OZONE_FILESYSTEM_SNAPSHOT_ENABLED_KEY,
990991
OMConfigKeys.OZONE_FILESYSTEM_SNAPSHOT_ENABLED_DEFAULT);
992+
fsSnapshotMaxLimit = configuration.getInt(
993+
OMConfigKeys.OZONE_OM_FS_SNAPSHOT_MAX_LIMIT,
994+
OMConfigKeys.OZONE_OM_FS_SNAPSHOT_MAX_LIMIT_DEFAULT);
991995
omSnapshotManager = new OmSnapshotManager(this);
992996

993997
// Snapshot metrics
@@ -4365,6 +4369,15 @@ public boolean isFilesystemSnapshotEnabled() {
43654369
return fsSnapshotEnabled;
43664370
}
43674371

4372+
/**
4373+
* Get the maximum number of Ozone filesystem snapshots allowed.
4374+
*
4375+
* @return the maximum number of Ozone filesystem snapshots allowed.
4376+
*/
4377+
public int getFsSnapshotMaxLimit() {
4378+
return fsSnapshotMaxLimit;
4379+
}
4380+
43684381
/**
43694382
* Get DB updates since a specific sequence number.
43704383
*

hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -165,6 +165,14 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, Execut
165165
throw new OMException("Snapshot already exists", FILE_ALREADY_EXISTS);
166166
}
167167

168+
// Check snapshot limit
169+
int maxSnapshots = ozoneManager.getFsSnapshotMaxLimit();
170+
if (omMetrics.getNumSnapshotActive() >= maxSnapshots) {
171+
throw new OMException(
172+
String.format("Snapshot limit of %d reached. Cannot create more snapshots.", maxSnapshots),
173+
OMException.ResultCodes.INVALID_SNAPSHOT_ERROR);
174+
}
175+
168176
// Note down RDB latest transaction sequence number, which is used
169177
// as snapshot generation in the Differ.
170178
final long dbLatestSequenceNumber =

hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotCreateRequest.java

Lines changed: 36 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
package org.apache.hadoop.ozone.om.request.snapshot;
1919

2020
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
21+
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_FS_SNAPSHOT_MAX_LIMIT_DEFAULT;;
2122
import static org.apache.hadoop.ozone.om.helpers.SnapshotInfo.getFromProtobuf;
2223
import static org.apache.hadoop.ozone.om.helpers.SnapshotInfo.getTableKey;
2324
import static org.apache.hadoop.ozone.om.request.OMRequestTestUtils.createSnapshotRequest;
@@ -28,7 +29,7 @@
2829
import static org.junit.jupiter.api.Assertions.assertNull;
2930
import static org.junit.jupiter.api.Assertions.assertThrows;
3031
import static org.junit.jupiter.api.Assertions.assertTrue;
31-
import static org.mockito.Mockito.any;
32+
import static org.mockito.ArgumentMatchers.any;
3233
import static org.mockito.Mockito.when;
3334

3435
import java.io.IOException;
@@ -69,6 +70,7 @@ public class TestOMSnapshotCreateRequest extends TestSnapshotRequestAndResponse
6970
public void setup() throws Exception {
7071
snapshotName1 = UUID.randomUUID().toString();
7172
snapshotName2 = UUID.randomUUID().toString();
73+
when(getOzoneManager().getFsSnapshotMaxLimit()).thenReturn(OZONE_OM_FS_SNAPSHOT_MAX_LIMIT_DEFAULT);
7274
}
7375

7476
@ValueSource(strings = {
@@ -264,6 +266,39 @@ public void testEntryExists() throws Exception {
264266
assertEquals(2, getOmMetrics().getNumSnapshotCreates());
265267
}
266268

269+
@Test
270+
public void testSnapshotLimit() throws Exception {
271+
when(getOzoneManager().isAdmin(any())).thenReturn(true);
272+
273+
when(getOzoneManager().getFsSnapshotMaxLimit()).thenReturn(1);
274+
275+
String key1 = getTableKey(getVolumeName(), getBucketName(), snapshotName1);
276+
277+
OMRequest omRequest =
278+
createSnapshotRequest(getVolumeName(), getBucketName(), snapshotName1);
279+
OMSnapshotCreateRequest omSnapshotCreateRequest = doPreExecute(omRequest);
280+
281+
assertNull(getOmMetadataManager().getSnapshotInfoTable().get(key1));
282+
omSnapshotCreateRequest.validateAndUpdateCache(getOzoneManager(), 1);
283+
284+
assertNotNull(getOmMetadataManager().getSnapshotInfoTable().get(key1));
285+
286+
// Should fail as snapshot limit is 1
287+
omRequest = createSnapshotRequest(getVolumeName(), getBucketName(), snapshotName2);
288+
omSnapshotCreateRequest = doPreExecute(omRequest);
289+
OMClientResponse omClientResponse =
290+
omSnapshotCreateRequest.validateAndUpdateCache(getOzoneManager(), 2);
291+
292+
OMResponse omResponse = omClientResponse.getOMResponse();
293+
assertNotNull(omResponse.getCreateSnapshotResponse());
294+
assertEquals(OzoneManagerProtocolProtos.Status.INVALID_SNAPSHOT_ERROR,
295+
omResponse.getStatus());
296+
297+
assertEquals(1, getOmMetrics().getNumSnapshotCreateFails());
298+
assertEquals(1, getOmMetrics().getNumSnapshotActive());
299+
assertEquals(2, getOmMetrics().getNumSnapshotCreates());
300+
}
301+
267302
private void renameKey(String fromKey, String toKey, long offset)
268303
throws IOException {
269304
OmKeyInfo toKeyInfo = addKey(toKey, offset + 1L);

0 commit comments

Comments
 (0)