Skip to content

Retry on segment cannot build#15234

Merged
chenboat merged 4 commits intoapache:masterfrom
lnbest0707:upsream_fork_retry
Mar 26, 2025
Merged

Retry on segment cannot build#15234
chenboat merged 4 commits intoapache:masterfrom
lnbest0707:upsream_fork_retry

Conversation

@lnbest0707
Copy link
Contributor

@lnbest0707 lnbest0707 commented Mar 10, 2025

ingestion feature

When mutable segments get transferred to immutable segments. There are a few preConditions check like

  • 4GB SV compressed column size
  • 2G MV num of elements
    Those check failure would raise IllegalStateException, set _state to error and then stop there forever. This is due to the reason that Pinot is assuming such build failures are transient and would eventually be fixed by other replicas. However, in most of cases, such failures are deterministic and would fail on all replicas.

This PR will introduce a new SegmentCannotBuildRequest between server and controller. Once Preconditions check failed and there's no other replicas able to build so far. The controller would reduce the segment size by half and reset the segments.
Call out a few corner scenarios, e.g. segment has 2 replicas A and B

  • "A" build failed and send SegmentCannotBuildRequest first, then reset segment will be sent immediately to both A and B. B would not try to build even if it may be successful in the future.
  • "A" build succeed and has sent "commitStart" request. Then B build failed and send SegmentCannotBuildRequest. B's request would be ignored. And eventually B would get the copy from A or deepstore.

Ingestion stop issue would no longer happen. The only side effect is that user might see some latest data now but then not be able to see it later temporarily. (AS we've reset the latest immutable segments)

@codecov-commenter
Copy link

codecov-commenter commented Mar 10, 2025

Codecov Report

Attention: Patch coverage is 41.26984% with 37 lines in your changes missing coverage. Please review.

Project coverage is 63.65%. Comparing base (59551e4) to head (4c17f2d).
Report is 1875 commits behind head on master.

Files with missing lines Patch % Lines
...er/api/resources/LLCSegmentCompletionHandlers.java 0.00% 10 Missing ⚠️
...a/manager/realtime/RealtimeSegmentDataManager.java 37.50% 8 Missing and 2 partials ⚠️
.../helix/core/realtime/SegmentCompletionManager.java 0.00% 7 Missing ⚠️
...altime/ServerSegmentCompletionProtocolHandler.java 0.00% 5 Missing ⚠️
...ot/common/protocols/SegmentCompletionProtocol.java 0.00% 2 Missing ⚠️
...ix/core/realtime/BlockingSegmentCompletionFSM.java 0.00% 2 Missing ⚠️
.../core/realtime/PinotLLCRealtimeSegmentManager.java 93.75% 0 Missing and 1 partial ⚠️
Additional details and impacted files
@@             Coverage Diff              @@
##             master   #15234      +/-   ##
============================================
+ Coverage     61.75%   63.65%   +1.90%     
- Complexity      207     1459    +1252     
============================================
  Files          2436     2782     +346     
  Lines        133233   156961   +23728     
  Branches      20636    24078    +3442     
============================================
+ Hits          82274    99910   +17636     
- Misses        44911    49534    +4623     
- Partials       6048     7517    +1469     
Flag Coverage Δ
custom-integration1 100.00% <ø> (+99.99%) ⬆️
integration 100.00% <ø> (+99.99%) ⬆️
integration1 100.00% <ø> (+99.99%) ⬆️
integration2 0.00% <ø> (ø)
java-11 63.58% <41.26%> (+1.87%) ⬆️
java-21 63.53% <41.26%> (+1.91%) ⬆️
skip-bytebuffers-false 63.64% <41.26%> (+1.90%) ⬆️
skip-bytebuffers-true 63.47% <41.26%> (+35.74%) ⬆️
temurin 63.65% <41.26%> (+1.90%) ⬆️
unittests 63.64% <41.26%> (+1.90%) ⬆️
unittests1 56.13% <39.28%> (+9.24%) ⬆️
unittests2 34.23% <31.74%> (+6.50%) ⬆️

Flags with carried forward coverage won't be shown. Click here to find out more.

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

🚀 New features to boost your workflow:
  • Test Analytics: Detect flaky tests, report on failures, and find test suite problems.
  • 📦 JS Bundle Analysis: Save yourself from yourself by tracking and limiting bundle sizes in JS merges.

prevSegmentZKMetadata.getStatus());

int prevTargetNumRows = prevSegmentZKMetadata.getSizeThresholdToFlushSegment();
int newNumRows = Math.min(prevNumRows / 2, prevTargetNumRows / 2);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is prevNumRows strictly smaller than prevTargetNumRows?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not always. The endCriteria check is between consuming each "batch" of messages. So it is possible that after consuming a batch, the numRowsIndexed goes slightly over threshold.

@chenboat chenboat self-requested a review March 18, 2025 23:15
Copy link
Contributor

@Jackie-Jiang Jackie-Jiang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This won't work for pauseless consumption because segment build happens after commit. cc @KKcorps @9aman

Ideally we should try to follow the mechanism introduced in #14479 to early commit the segment when it cannot consume more rows

@lnbest0707
Copy link
Contributor Author

This won't work for pauseless consumption because segment build happens after commit. cc @KKcorps @9aman

Ideally we should try to follow the mechanism introduced in #14479 to early commit the segment when it cannot consume more rows

I have a follow up PR for early commit in #15120. However, the threshold is not easy to set. Concerns are listed in that PR.
This retry mechanism (enabled by the table config) could be preferred in some cases. E.g. for large tables, false alarm on early commits is not always acceptable(creating too many segments).

Regarding to pauseless consumption, this issue could also arise and be even more severe(no way to recover). In my opinion, we should introduce similar thing in the /reingest failure recovery loop introduced in #14920.

}

/**
* An instance is reporting that it cannot build segment due to non-recoverable error, usually due to size too large.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When one server fails to build the segment and the controller updates the segment size target to half of the previous value, how will the other replica servers handle such change? Will they continue to build? There are two cases: (1)if they also have the same build failure, they will reset their target? (2) What happen if they build the segments successfully?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When one segment build failed and the other has not committed (either succeed or fail), controller would reduce the size and reset the segment. This would force to destroy the mutable segment and reingest.
When one segment build failed but the other has already succeed and commit, controller will drop the request and allow the succeed one completes its build. The failed one will eventually be replaced by the one built by the succeed one.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also updated the PR description with same.

@Jackie-Jiang
Copy link
Contributor

Regarding to pauseless consumption, this issue could also arise and be even more severe(no way to recover). In my opinion, we should introduce similar thing in the /reingest failure recovery loop introduced in #14920.

For pauseless consumption, it will be hard to reingest the segment with a different size threshold because start/end offset are already determined. We need to detect the issue earlier, i.e. before segment build

@lnbest0707
Copy link
Contributor Author

Regarding to pauseless consumption, this issue could also arise and be even more severe(no way to recover). In my opinion, we should introduce similar thing in the /reingest failure recovery loop introduced in #14920.

For pauseless consumption, it will be hard to reingest the segment with a different size threshold because start/end offset are already determined. We need to detect the issue earlier, i.e. before segment build

True.. The current limitation is on the segment sequence number. Once the segment offset is decided, when we reingest it, it is hard to split it into 2 as the next continuous sequence number is already taken. Probably some tricks need to be introduced there.

@chenboat chenboat merged commit 3bcca12 into apache:master Mar 26, 2025
22 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants