-
Notifications
You must be signed in to change notification settings - Fork 1.3k
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
Storage Quota imposition on Realtime tables #13584
Storage Quota imposition on Realtime tables #13584
Conversation
Codecov ReportAttention: Patch coverage is
Additional details and impacted files@@ Coverage Diff @@
## master #13584 +/- ##
============================================
- Coverage 61.75% 57.93% -3.82%
- Complexity 207 219 +12
============================================
Files 2436 2613 +177
Lines 133233 143294 +10061
Branches 20636 22004 +1368
============================================
+ Hits 82274 83021 +747
- Misses 44911 53766 +8855
- Partials 6048 6507 +459
Flags with carried forward coverage won't be shown. Click here to find out more. ☔ View full report in Codecov by Sentry. |
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.
Mostly good. Can we add a test to verify this?
@swaminathanmanish @sajjad-moradi please also take a look
ZNRecord znRecord = idealState.getRecord(); | ||
znRecord.setSimpleField(IS_QUOTA_EXCEEDED, Boolean.valueOf(quotaExceeded).toString()); | ||
return new IdealState(znRecord); | ||
}, RetryPolicies.noDelayRetryPolicy(1)); |
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.
Why not allow retry here?
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.
No strong reason to not retry. Was keeping it inline with the table pause IS update.
Right now in case of failure the next segment commit or validation job, whichever runs first, will update the IS.
Let me know if you have any magic number for retry in mind, maybe 3?
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.
If we do retry, we will have to move this check also inside the updater lambda right
IdealState is = getIdealState(tableNameWithType);
if (is.getRecord().getBooleanField(IS_QUOTA_EXCEEDED, false) != quotaExceeded)
...r/src/main/java/org/apache/pinot/controller/validation/RealtimeSegmentValidationManager.java
Outdated
Show resolved
Hide resolved
pinot-controller/src/main/java/org/apache/pinot/controller/validation/StorageQuotaChecker.java
Outdated
Show resolved
Hide resolved
pinot-controller/src/main/java/org/apache/pinot/controller/validation/StorageQuotaChecker.java
Show resolved
Hide resolved
Will review next week. |
...ain/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java
Outdated
Show resolved
Hide resolved
ZNRecord znRecord = idealState.getRecord(); | ||
znRecord.setSimpleField(IS_QUOTA_EXCEEDED, Boolean.valueOf(quotaExceeded).toString()); | ||
return new IdealState(znRecord); | ||
}, RetryPolicies.noDelayRetryPolicy(1)); |
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.
If we do retry, we will have to move this check also inside the updater lambda right
IdealState is = getIdealState(tableNameWithType);
if (is.getRecord().getBooleanField(IS_QUOTA_EXCEEDED, false) != quotaExceeded)
...ain/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java
Outdated
Show resolved
Hide resolved
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.
LGTM
@sajjad-moradi can you please review the PR? |
I just had a discussion with @mcvsubbu about this. Here's what we think: "pauseStatus": {
"isPaused": true,
"reasonCode": "STORAGE_QUOTA_EXCEEDED",
"comment": "Current storage 2.3G exceeds quota 2.0G",
"time": "2024-08-01T09:30:00Z"
} And here is an example for a regular pause that's caused by an admin hitting the pause endpoint: "pauseStatus": {
"isPaused": true,
"reasonCode": "ADMINISTRATIVE",
"comment": "Need to change the underlying topic",
"time": "2024-08-01T09:30:00Z"
} This way in case in future there's another reason for pausing the table, there's no need to add yet another field to IS to pause the table. We can simply do that by adding another |
@sajjad-moradi @mcvsubbu , the
Here the priority is well established so may not be a big concern. |
The priority you described makes sense. Basically storage quota should not be even checked if the table is paused by an admin. |
On a separate note, @mcvsubbu thinks it's better to have all the logic for hanlding storage quota (pausing/resuming) in one place, in "realtime segment validation manager" periodic job. |
@Jackie-Jiang any thoughts here? |
There is enough going on during the segment completion, that I would hate to add more logic there with more conditions. Can you state why you need the quota to be checked during segment completion? I am even fine adding logic in the background task to remove latest segments (and adjust consumption offsets, of course) if the quota is exceeded (it may matter in some cases that the excess quota will result in charges). On the other hand, if this is the case, the metric on current storage use should be monitored in order to flag such situations early and in advance |
Hey @mcvsubbu @sajjad-moradi , it is ok if the table storage spills over the quota by few segments before stopping the consumption and that said we can keep the quota validation + IS update logic at one place in segment validation manager.
There is no strict storage quota imposition requirement, but also didn’t saw any concern in putting the check during segment completion
We don’t want to do this as this will affect the query results (moving back in time). If it turns out that segment validation manager job frequency is not working for the storage quota imposition we can introduce a separate job when needed. |
pinot-common/src/main/java/org/apache/pinot/common/metrics/ControllerGauge.java
Outdated
Show resolved
Hide resolved
ec9e673
to
17ae277
Compare
17ae277
to
623f3b1
Compare
4620372
to
95659c8
Compare
Hey @mcvsubbu @sajjad-moradi I have updated the PR with the changes from PauseState PR and also removed the storage quota check from segment commit process and now only have it as part of the |
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.
Thanks for making all the changes.
_tableSizeReader = tableSizeReader; | ||
_controllerMetrics = controllerMetrics; | ||
_leadControllerManager = leadControllerManager; | ||
_pinotHelixResourceManager = pinotHelixResourceManager; | ||
_controllerConf = controllerConf; |
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.
Nit. I may just pick up whatever is needed from config and not keep the entire config object here. In this case, an _isEnabled
boolean will suffice?
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.
Makes sense, will make the change.
@@ -43,14 +44,19 @@ public class StorageQuotaChecker { | |||
private final ControllerMetrics _controllerMetrics; | |||
private final LeadControllerManager _leadControllerManager; | |||
private final PinotHelixResourceManager _pinotHelixResourceManager; | |||
private final ControllerConf _controllerConf; | |||
private final int _timeout; |
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 think it is useful to call it _timeoutMs
since it improves readability.
_llcRealtimeSegmentManager.ensureAllPartitionsConsuming(tableConfig, streamConfig, | ||
context._recreateDeletedConsumingSegment, context._offsetCriteria); | ||
context._recreateDeletedConsumingSegment || !pauseState.isPaused(), context._offsetCriteria); |
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.
Please change this to not call ensureAllPartitionsConsuming
if the table is paused (or storage is exceeeded) rather than overloading the paused boolean with recreation of deleted segment.
// Skips updating the pause state if table is paused by admin | ||
PauseState pauseState = computePauseState(tableNameWithType); | ||
|
||
if (!pauseState.isPaused()) { |
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.
Sorry for the nit, but this reads like it is invoking the periodic consumption fix if the table is not paused. That is partially true, but another factor is that the table may have run out of quota.
So, it is better to create a method like shouldEnsurePartitionsConsuming()
from which we return a boolean instead of returning the PauseState.
Thanks.
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.
Had a chat offline, will handle the future of _recreateDeletedConsumingSegment
flag in a separate PR.
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 learned that the flag is not quite as I had imagined. Yes, it is a config, and if turned true, then it ends up invoking the periodic job with 'true' (which ends up handling a condition of someone accidentally or intentionally removing the consuming segments of a table).
the flag is also used in the "resume" part because resume happens via kicking off the periodic task.
cc: @sajjad-moradi
boolean unPausedUponStorageWithinQuota = | ||
pauseState.getReasonCode().equals(PauseState.ReasonCode.STORAGE_QUOTA_EXCEEDED); | ||
_llcRealtimeSegmentManager.ensureAllPartitionsConsuming(tableConfig, streamConfig, | ||
context._recreateDeletedConsumingSegment || unPausedUponStorageWithinQuota, context._offsetCriteria); |
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.
Let us re-factor this as follows:
- Create a method called
shouldRestartConsumption()
that returns a boolean - Change the original call to
ensureConsuming
to be under an if check based on the return of this method - The method
shouldRestartConsumption()
returns false if either one of the conditions hold: (a) the table has been paused by admin (2) the table is out of quota
Agree?
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.
what about _recreateDeletedConsumingSegment
?
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.
Leave it as it is. That variable is not needed to evaluate shouldRestartConsumption()
(actually, a better name could be shouldEnsureConsuming()
). If the condition evaluates to true, then call it the same way as before.
pauseState.getReasonCode().equals(PauseState.ReasonCode.STORAGE_QUOTA_EXCEEDED); | ||
if (unPausedUponStorageWithinQuota) { | ||
// recreate consuming segments if table is resumed upon the table storage getting within quota limit | ||
context._recreateDeletedConsumingSegment = true; |
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.
context._recreateDeletedConsumingSegment = true; | |
return true; |
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.
Once the quota is increased on table or storage is freed due to retention, how will the consuming segments be recreated unless we set this flag?
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.
Fair enough. I was thinking of the world when the flag is not there. Maybe it is good to remove the flag in this PR. Alternatively, you can commit this and either one of us can remove the flag.
here is what I have for the flag removal:
- change ensureConsuming() method to be in favor of the flag always being true
- Remove the flag from the config and the map. Anyone who deletes the consuming segments by mistake will want to restart consumption, so it will work. Anyone who deliberately wants to discard consuming segments can do with the pause consumption operation command we have. If necessary, we can enhance the pause command to optionally discard current consuming segments.
I will give a ship it, let me know if you want me to raise a PR to remove that flag.
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.
No problem, I'll raise a PR to remove the flag
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.
Description
This PR allows pinot to impose storage quota restrictions on realtime tables.
To block the ingestion on realtime tables below consideration are kept in mind
Also to keep it simple the ingestion is blocked only when storage quotas are breached
Blocking ingestion
RealtimeSegmentValidationManager
controller job.PauseState
is updated accordingly by setting pausereasonCode
asSTORAGE_QUOTA_EXCEEDED
on the table ISWhat if the
pauseState
is updated manually on zk directly to resume the table ingestion?Updating the
pauseState
alone will not create the new consuming segments. New consuming segments can be created by:/resumeConsumption
API on controllerRealtimeSegmentValidationManager
job with settingrecreateDeletedConsumingSegment
astrue
The resume consumption API internally depends on the
RealtimeSegmentValidationManager
task itself to handle the consuming segment creation.As the storage quota validation is itself part of the
RealtimeSegmentValidationManager
job it will again check for the storage quota and set thepauseState
to the right value.Resuming ingestion
Once a table exceeds the storage quota, we may need to resume the consumption in below cases:
To automate the resume consumption flow we will depend on the
RealtimeSegmentValidationManager
job itself.In case the user does not want to wait for the periodic job run, they can manually trigger the job from the controller API or use the resume consumption endpoint.