-
Notifications
You must be signed in to change notification settings - Fork 1.4k
Allows segments deletion in build for pauseless tables #15299
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
Allows segments deletion in build for pauseless tables #15299
Conversation
Codecov ReportAttention: Patch coverage is
Additional details and impacted files@@ Coverage Diff @@
## master #15299 +/- ##
============================================
+ Coverage 61.75% 63.19% +1.43%
- Complexity 207 1375 +1168
============================================
Files 2436 2812 +376
Lines 133233 158931 +25698
Branches 20636 24339 +3703
============================================
+ Hits 82274 100430 +18156
- Misses 44911 50910 +5999
- Partials 6048 7591 +1543
Flags with carried forward coverage won't be shown. Click here to find out more. ☔ View full report in Codecov by Sentry. 🚀 New features to boost your workflow:
|
| + "The retention period controls how long deleted segments are retained before permanent removal. " | ||
| + "It follows this precedence: input parameter → table config → cluster setting → 7d default. " | ||
| + "Use 0d or -1d for immediate deletion without retention.") | ||
| public SuccessResponse deletePauselessSegments( |
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.
API should be named better. Like reconcile pauseless table
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.
It's not reconciling right ?
It's more of a way to manually reset the table to a stable state. It requires too much manual intervention to call it reconciliation.
It basically is deleting from the oldest segment for that partition and does not care about the state of the table.
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.
How about pruneSegmentsFromSequence or truncatePartitionSegments
36a7b1e to
fd82865
Compare
| Map<Integer, LLCSegmentName> partitionToOldestSegment = new HashMap<>(); | ||
|
|
||
| for (String segment : segments) { | ||
| LLCSegmentName llcSegmentName = new LLCSegmentName(segment); |
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's add NPE check 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.
Also a minor same table name check
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's add NPE check here
I have thrown an exception in case this segment name passed is invalid.
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.
Also a minor same table name check
Done. Thanks for pointing this out.
I have updated the UT's to reflect to this.
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.
Can we have some check in Validation job to atleast automate alerting part
...ler/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java
Show resolved
Hide resolved
| Map<String, Map<String, String>> segmentsToInstanceState = idealState.getRecord().getMapFields(); | ||
|
|
||
| for (String segmentName : segmentsToInstanceState.keySet()) { | ||
| LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName); |
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.
Add a check here if segment name doesn't fit into to LLC segment name scheme. It can happen if it was uploaded by some task or user manually.
we should skip that segment
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 thought of throwing exception in this case. Do you feel customers usually do this for realtime tables as well ?
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.
yeah, i have seen instances in our envs
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.
Also, it it's an uploaded segment how do we know the sequence id for the same.
|
Since we are deleting even the latest consuming segment, are we relying on RealtimeSegmentValidationManager to create new ones after deletion? I feel like we should trigger it by default in this API. |
I think these two operations should not be clubbed together. The rationale for this is that this is an API for DR and that should be a rare scenario. The user can the trigger RealtimeValidationManager. |
aee5f5b to
3fd8332
Compare
Context
A segment is marked ONLINE before it is build for pauseless tables. This increases the chances of data missing in the middle e.g.
seg 0 : Build and stored on the segment
seg 1: Ingested but build failed
seg 2: Ingest and build succeeded
The gap created in the data due to absence of seg 1 is filled by DR: #14920
For Dedup and Partial Upserts, Segment 2 may contain incorrect data because it processed without having Segment 1's updates
Proposed Solution
This PR introduces a new API that enables deleting segments from pauseless-enabled tables. For each specified segment, it deletes that segment and all segments with higher sequence IDs in the same partition.
An optional force flag to bypass pauseless and table state checks.

Testing