-
Notifications
You must be signed in to change notification settings - Fork 588
HDDS-12596. OM fs snapshot max limit is not enforced #8157
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 20 commits
a4d878e
f14a095
fe3780c
39d87be
8338f29
0834515
3c2727d
7b52f1a
2607c3e
78bc4c4
3b367e4
902b507
65922fb
3cb97fe
0d2d15b
8f0bab2
6465e9f
385eeb1
ede525e
55cb418
1106481
be31e15
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,6 +24,8 @@ | |
| import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_CHECKPOINT_DIR; | ||
| import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIFF_DB_NAME; | ||
| import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_INDICATOR; | ||
| import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_FS_SNAPSHOT_MAX_LIMIT; | ||
| import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_FS_SNAPSHOT_MAX_LIMIT_DEFAULT; | ||
| import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SNAPSHOT_CACHE_CLEANUP_SERVICE_RUN_INTERVAL; | ||
| import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SNAPSHOT_CACHE_CLEANUP_SERVICE_RUN_INTERVAL_DEFAULT; | ||
| import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE; | ||
|
|
@@ -63,6 +65,8 @@ | |
| import java.util.Set; | ||
| import java.util.UUID; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import java.util.concurrent.atomic.AtomicReference; | ||
| import java.util.stream.Collectors; | ||
| import org.apache.commons.lang3.tuple.Pair; | ||
| import org.apache.hadoop.hdds.StringUtils; | ||
|
|
@@ -173,6 +177,9 @@ public final class OmSnapshotManager implements AutoCloseable { | |
| // Soft limit of the snapshot cache size. | ||
| private final int softCacheSize; | ||
|
|
||
| private int fsSnapshotMaxLimit; | ||
| private final AtomicInteger inFlightSnapshotCount = new AtomicInteger(0); | ||
hemantk-12 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| public OmSnapshotManager(OzoneManager ozoneManager) { | ||
|
|
||
| boolean isFilesystemSnapshotEnabled = | ||
|
|
@@ -247,6 +254,9 @@ public OmSnapshotManager(OzoneManager ozoneManager) { | |
| this.softCacheSize = ozoneManager.getConfiguration().getInt( | ||
| OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE, | ||
| OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE_DEFAULT); | ||
|
|
||
| fsSnapshotMaxLimit = ozoneManager.getConfiguration().getInt(OZONE_OM_FS_SNAPSHOT_MAX_LIMIT, | ||
| OZONE_OM_FS_SNAPSHOT_MAX_LIMIT_DEFAULT); | ||
|
|
||
| CacheLoader<UUID, OmSnapshot> loader = createCacheLoader(); | ||
|
|
||
|
|
@@ -860,6 +870,42 @@ private void validateSnapshotsExistAndActive(final String volumeName, | |
| checkSnapshotActive(toSnapInfo, false); | ||
| } | ||
|
|
||
| public void snapshotLimitCheck() throws IOException, OMException { | ||
| OmMetadataManagerImpl omMetadataManager = (OmMetadataManagerImpl) ozoneManager.getMetadataManager(); | ||
| SnapshotChainManager snapshotChainManager = omMetadataManager.getSnapshotChainManager(); | ||
| int currentSnapshotNum = snapshotChainManager.getGlobalSnapshotChain().size(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @peterxcli Can't we call this inside inflightSnapshotCount.updateAndGet() function that would make this threadsafe and consistent.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I was off the previous week so couldn't review this PR in time. Can you please make this change in a follow up jira.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sure and thanks for the review! Created HDDS-12952 for this. |
||
|
|
||
| AtomicReference<OMException> exceptionRef = new AtomicReference<>(null); | ||
| inFlightSnapshotCount.updateAndGet(count -> { | ||
| if (currentSnapshotNum + count >= fsSnapshotMaxLimit) { | ||
| exceptionRef.set(new OMException( | ||
| String.format("Snapshot limit of %d reached. Cannot create more snapshots. " + | ||
| "Current snapshots: %d, In-flight creations: %d", | ||
| fsSnapshotMaxLimit, currentSnapshotNum, count) + | ||
| " If you already deleted some snapshots, " + | ||
| "please wait for the background service to complete the cleanup.", | ||
| OMException.ResultCodes.TOO_MANY_SNAPSHOTS)); | ||
| return count; | ||
| } | ||
| return count + 1; | ||
| }); | ||
| if (exceptionRef.get() != null) { | ||
| throw exceptionRef.get(); | ||
| } | ||
| } | ||
|
|
||
| public void decrementInFlightSnapshotCount() { | ||
| inFlightSnapshotCount.decrementAndGet(); | ||
| } | ||
|
|
||
| public void resetInFlightSnapshotCount() { | ||
| inFlightSnapshotCount.set(0); | ||
| } | ||
|
|
||
| public int getInFlightSnapshotCount() { | ||
| return inFlightSnapshotCount.get(); | ||
| } | ||
|
|
||
| private int getIndexFromToken(final String token) throws IOException { | ||
| if (isBlank(token)) { | ||
| return 0; | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I have check that if I comment out the reset logic, this test would failed with: