Skip to content
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

Supports Force Committing Segments in Batches #14811

Open
wants to merge 54 commits into
base: master
Choose a base branch
from

Conversation

noob-se7en
Copy link
Contributor

@noob-se7en noob-se7en commented Jan 14, 2025

Problem Statement
The Force Commit API can cause high ingestion lag and slower queries because it triggers the simultaneous segment commit for all consuming segments. This happens because:

  1. If N is the number of partition groups a server is consuming from, The API will cause all of the N consuming segments to commit. Hence N consumer threads will rush to acquire segment build semaphore. If the Semaphore allows only M permits, Only M consuming segments are in the segment build stage and the remaining (N - M) consumer threads are waiting on the Semaphore. Since (N - M) consumer threads are waiting, the consumption lag can become substantial.
  2. Since M consuming segments are built in parallel, queries can become slower on poorly sized servers due to high memory consumption.

Solution
Adds an additional optional Integer parameter: batchSize to the forceCommit API (Default Value = Integer.MAX_VALUE i.e. commit as many segments as possible at once if no batchSize is provided) .
Screenshot 2025-01-18 at 1 43 41 AM

@codecov-commenter
Copy link

codecov-commenter commented Jan 14, 2025

Codecov Report

Attention: Patch coverage is 52.57732% with 46 lines in your changes missing coverage. Please review.

Project coverage is 63.75%. Comparing base (59551e4) to head (b95a2f6).
Report is 1593 commits behind head on master.

Files with missing lines Patch % Lines
.../core/realtime/PinotLLCRealtimeSegmentManager.java 64.55% 26 Missing and 2 partials ⚠️
...ller/api/resources/PinotRealtimeTableResource.java 0.00% 18 Missing ⚠️
Additional details and impacted files
@@             Coverage Diff              @@
##             master   #14811      +/-   ##
============================================
+ Coverage     61.75%   63.75%   +2.00%     
- Complexity      207     1612    +1405     
============================================
  Files          2436     2708     +272     
  Lines        133233   151313   +18080     
  Branches      20636    23364    +2728     
============================================
+ Hits          82274    96475   +14201     
- Misses        44911    47596    +2685     
- Partials       6048     7242    +1194     
Flag Coverage Δ
custom-integration1 100.00% <ø> (+99.99%) ⬆️
integration 100.00% <ø> (+99.99%) ⬆️
integration1 100.00% <ø> (+99.99%) ⬆️
integration2 0.00% <ø> (ø)
java-11 63.71% <52.57%> (+2.00%) ⬆️
java-21 63.65% <52.57%> (+2.02%) ⬆️
skip-bytebuffers-false 63.73% <52.57%> (+1.98%) ⬆️
skip-bytebuffers-true 63.63% <52.57%> (+35.90%) ⬆️
temurin 63.75% <52.57%> (+2.00%) ⬆️
unittests 63.75% <52.57%> (+2.00%) ⬆️
unittests1 56.32% <ø> (+9.43%) ⬆️
unittests2 34.07% <52.57%> (+6.34%) ⬆️

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.

@Jackie-Jiang
Copy link
Contributor

This solves #11950

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.

Seems you are pushing the batch throttling to the server side. What will happen if some replicas decide to commit, and others get throttled? Even worse, could this cause deadlock?

@siddharthteotia
Copy link
Contributor

Is it not possible to solve the problem on controller / coordinate from the controller ? Pushing this down to the individual server will likely lead to error-prone situations

@noob-se7en
Copy link
Contributor Author

@Jackie-Jiang I don't quite get what is meant by

and others get throttled?

Regarding Deadlock or any edge case - Server will use the same logic which is used /tables/forceCommitStatus/{jobId} to check the status of the batch, so there should be no deadlock.

@noob-se7en noob-se7en marked this pull request as ready for review January 16, 2025 21:24
sendForceCommitMessageToServers(tableNameWithType, targetConsumingSegments);

List<Set<String>> segmentBatchList = getSegmentBatchList(idealState, targetConsumingSegments, batchSize);
ExecutorService executorService = Executors.newFixedThreadPool(1);
Copy link
Contributor

Choose a reason for hiding this comment

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

Do not create new thread pool on every request

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ohh right!
Fixed (created a separate bounded ExecutorService for forceCommitAPI)

sendForceCommitMessageToServers(tableNameWithType, segmentBatchToCommit);

try {
Thread.sleep(FORCE_COMMIT_STATUS_CHECK_INTERVAL_MS);
Copy link
Contributor

Choose a reason for hiding this comment

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

Why sleep at start?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This sleep is to save zookeeper calls, as at this point of code it's almost certain that all segments are yet to be committed.
(we need a new retryPolicy with initial start delay)

String state = instanceToStateMap.get(instance);
if (state.equals(SegmentStateModel.CONSUMING)) {
instanceToConsumingSegments.putIfAbsent(instance, new LinkedList<>());
instanceToConsumingSegments.get(instance).add(segmentName);
Copy link
Contributor

@KKcorps KKcorps Jan 17, 2025

Choose a reason for hiding this comment

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

You can simply this by using compute instead of putIfAbsent and get

List<Set<String>> segmentBatchList =
realtimeSegmentManager.getSegmentBatchList(idealState, targetConsumingSegment, 2);

// i1 = [seg0, seg4, seg5, seg6]
Copy link
Contributor

Choose a reason for hiding this comment

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

remove these comments

@@ -170,19 +169,28 @@ public Map<String, String> forceCommit(
@ApiParam(value = "Comma separated list of partition group IDs to be committed") @QueryParam("partitions")
String partitionGroupIds,
@ApiParam(value = "Comma separated list of consuming segments to be committed") @QueryParam("segments")
String consumingSegments, @Context HttpHeaders headers) {
String consumingSegments,
@ApiParam(value = "Max number of consuming segments to commit at once") @QueryParam("batchSize")
Copy link
Contributor

Choose a reason for hiding this comment

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

mention that default is all consuming segments at once in description

@KKcorps
Copy link
Contributor

KKcorps commented Jan 17, 2025

Please update PR description to reflect the change in API signature

@noob-se7en noob-se7en requested a review from KKcorps January 17, 2025 20:14
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