-
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
Supports Force Committing Segments in Batches #14811
base: master
Are you sure you want to change the base?
Conversation
Codecov ReportAttention: Patch coverage is
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
Flags with carried forward coverage won't be shown. Click here to find out more. ☔ View full report in Codecov by Sentry. |
This solves #11950 |
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.
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?
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 |
@Jackie-Jiang I don't quite get what is meant by
Regarding Deadlock or any edge case - Server will use the same logic which is used |
… into add_batch_force_commit
… into add_batch_force_commit
… into add_batch_force_commit
… into add_batch_force_commit
sendForceCommitMessageToServers(tableNameWithType, targetConsumingSegments); | ||
|
||
List<Set<String>> segmentBatchList = getSegmentBatchList(idealState, targetConsumingSegments, batchSize); | ||
ExecutorService executorService = Executors.newFixedThreadPool(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.
Do not create new thread pool on every request
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.
ohh right!
Fixed (created a separate bounded ExecutorService for forceCommitAPI)
sendForceCommitMessageToServers(tableNameWithType, segmentBatchToCommit); | ||
|
||
try { | ||
Thread.sleep(FORCE_COMMIT_STATUS_CHECK_INTERVAL_MS); |
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 sleep at start?
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.
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); |
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.
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] |
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.
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") |
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.
mention that default is all consuming segments at once in description
Please update PR description to reflect the change in API signature |
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:
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) .