|
18 | 18 | package org.apache.hadoop.ozone.om.request.snapshot; |
19 | 19 |
|
20 | 20 | 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;; |
21 | 22 | import static org.apache.hadoop.ozone.om.helpers.SnapshotInfo.getFromProtobuf; |
22 | 23 | import static org.apache.hadoop.ozone.om.helpers.SnapshotInfo.getTableKey; |
23 | 24 | import static org.apache.hadoop.ozone.om.request.OMRequestTestUtils.createSnapshotRequest; |
|
28 | 29 | import static org.junit.jupiter.api.Assertions.assertNull; |
29 | 30 | import static org.junit.jupiter.api.Assertions.assertThrows; |
30 | 31 | import static org.junit.jupiter.api.Assertions.assertTrue; |
31 | | -import static org.mockito.Mockito.any; |
| 32 | +import static org.mockito.ArgumentMatchers.any; |
32 | 33 | import static org.mockito.Mockito.when; |
33 | 34 |
|
34 | 35 | import java.io.IOException; |
@@ -69,6 +70,7 @@ public class TestOMSnapshotCreateRequest extends TestSnapshotRequestAndResponse |
69 | 70 | public void setup() throws Exception { |
70 | 71 | snapshotName1 = UUID.randomUUID().toString(); |
71 | 72 | snapshotName2 = UUID.randomUUID().toString(); |
| 73 | + when(getOzoneManager().getFsSnapshotMaxLimit()).thenReturn(OZONE_OM_FS_SNAPSHOT_MAX_LIMIT_DEFAULT); |
72 | 74 | } |
73 | 75 |
|
74 | 76 | @ValueSource(strings = { |
@@ -264,6 +266,39 @@ public void testEntryExists() throws Exception { |
264 | 266 | assertEquals(2, getOmMetrics().getNumSnapshotCreates()); |
265 | 267 | } |
266 | 268 |
|
| 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 | + |
267 | 302 | private void renameKey(String fromKey, String toKey, long offset) |
268 | 303 | throws IOException { |
269 | 304 | OmKeyInfo toKeyInfo = addKey(toKey, offset + 1L); |
|
0 commit comments