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

[7338] Allow Reloading Segments with Multiple Threads #7893

Merged
merged 24 commits into from Dec 25, 2021

Conversation

suddendust
Copy link
Contributor

Description

This change allows users to reload segments of a table in parallel. More description here.

Upgrade Notes

Does this PR prevent a zero down-time upgrade? (Assume upgrade order: Controller, Broker, Server, Minion)

  • Yes

Does this PR fix a zero-downtime upgrade introduced earlier?

  • Yes

Does this PR otherwise need attention when creating release notes? Things to consider:

  • Yes

@suddendust suddendust marked this pull request as draft December 12, 2021 11:57
@suddendust
Copy link
Contributor Author

Working on tests for this.

@codecov-commenter
Copy link

codecov-commenter commented Dec 12, 2021

Codecov Report

Merging #7893 (b486e5a) into master (fb12a50) will decrease coverage by 0.02%.
The diff coverage is 76.31%.

Impacted file tree graph

@@             Coverage Diff              @@
##             master    #7893      +/-   ##
============================================
- Coverage     71.34%   71.31%   -0.03%     
- Complexity     4087     4194     +107     
============================================
  Files          1587     1595       +8     
  Lines         82071    82568     +497     
  Branches      12267    12320      +53     
============================================
+ Hits          58553    58885     +332     
- Misses        19550    19708     +158     
- Partials       3968     3975       +7     
Flag Coverage Δ
integration1 29.02% <76.31%> (-0.26%) ⬇️
integration2 27.66% <71.05%> (+0.01%) ⬆️
unittests1 68.23% <0.00%> (-0.10%) ⬇️
unittests2 14.29% <0.00%> (-0.15%) ⬇️

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

Impacted Files Coverage Δ
...server/starter/helix/HelixInstanceDataManager.java 81.50% <69.23%> (+1.02%) ⬆️
...pache/pinot/core/util/SegmentRefreshSemaphore.java 72.22% <72.22%> (ø)
...er/starter/helix/SegmentMessageHandlerFactory.java 73.68% <100.00%> (+1.08%) ⬆️
...data/manager/realtime/DefaultSegmentCommitter.java 0.00% <0.00%> (-80.00%) ⬇️
...ry/optimizer/statement/JsonStatementOptimizer.java 0.00% <0.00%> (-77.64%) ⬇️
.../pinot/spi/exception/BadQueryRequestException.java 66.66% <0.00%> (-33.34%) ⬇️
...a/manager/realtime/RealtimeSegmentDataManager.java 50.00% <0.00%> (-25.00%) ⬇️
...in/stream/kafka20/KafkaPartitionLevelConsumer.java 66.66% <0.00%> (-20.00%) ⬇️
...er/api/resources/LLCSegmentCompletionHandlers.java 43.56% <0.00%> (-18.82%) ⬇️
...data/manager/realtime/SegmentCommitterFactory.java 88.23% <0.00%> (-11.77%) ⬇️
... and 95 more

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update fb12a50...b486e5a. Read the comment docs.

Comment on lines 241 to 252
CompletableFuture.runAsync(() -> {
try {
reloadSegment(tableNameWithType, segmentMetadata, tableConfig, schema, forceDownload);
} catch (Exception e) {
String segmentName = segmentMetadata.getName();
LOGGER.error("Caught exception while reloading segment: {} in table: {}", segmentName, tableNameWithType, e);
failedSegments.add(segmentName);
sampleException.set(e);
} finally {
latch.countDown();
}
}, workers);
Copy link
Member

Choose a reason for hiding this comment

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

I think you can make better use of the CompletableFuture API here:

CompletableFuture.allOf(segmentsMetadata.stream()
.map(segmentMetadata -> CompletableFuture.runAsync(() -> {... reload the segment ...}, workers)
.toArray(CompletableFuture[]::new))
.get(timeout, TimeUnit.MILLISECONDS);

Which would eliminate the latch and make the code easier to reason about.

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 is excellent. Thanks! Definitely need to revisit CompletableFuture's API once.

@@ -49,6 +50,13 @@ public SegmentReloadMessage(@Nonnull String tableNameWithType, @Nullable String
znRecord.setBooleanField(FORCE_DOWNLOAD_KEY, forceDownload);
}

public SegmentReloadMessage(@Nonnull String tableNameWithType, @Nullable String segmentName, boolean forceDownload,
int parallelism) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I know this is draft PR

Just curious on the approach. How does the user specify parallelism parameter ? Can we use semaphore like we do in refresh (which is also delivered to server via helix message) ?

Copy link
Contributor

Choose a reason for hiding this comment

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

@suddendust if this is a draft PR, can you please add WIP in the title? thanks

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@siddharthteotia After some discussion, the conclusion is that there is no point in having yet another param to control the parallelism. Instead, using the existing _refreshThreadsSemaphore for this makes sense as reload and refresh are so similar. For each available permit, we reload a segment asynchronously.

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes, sound good

@suddendust suddendust changed the title [7338] Allow Reloading Segments with Multiple Threads [7338] [WIP] Allow Reloading Segments with Multiple Threads Dec 15, 2021
@suddendust suddendust marked this pull request as ready for review December 21, 2021 19:16
@suddendust suddendust changed the title [7338] [WIP] Allow Reloading Segments with Multiple Threads [7338] Allow Reloading Segments with Multiple Threads Dec 21, 2021
@suddendust
Copy link
Contributor Author

@Jackie-Jiang Requesting review, thanks!

try {
acquireSema("ALL", refreshThreadSemaphore);
Copy link
Contributor

Choose a reason for hiding this comment

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

Put the segment name

Suggested change
acquireSema("ALL", refreshThreadSemaphore);
acquireSema(segmentsMetadata.getName(), refreshThreadSemaphore);

try {
acquireSema("ALL", refreshThreadSemaphore);
Copy link
Contributor

Choose a reason for hiding this comment

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

acquireSema() should be outside of the try block

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, that was a miss. Thanks!

}
}
}, workers)).toArray(CompletableFuture[]::new)).get();
Copy link
Contributor

Choose a reason for hiding this comment

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

Do you need to shut down the executor service after it is done?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It should be GCed after ~ 60s (TTL of cached threads). However, it's a good idea to shut it down explicitly. Addressed.

LOGGER.info("Reloaded all segments in table: {}", tableNameWithType);
}

private void acquireSema(String context, Semaphore refreshThreadSemaphore)
Copy link
Contributor

Choose a reason for hiding this comment

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

The first argument is segmentName

(optional) This part is duplicated. One way to avoid duplicate code is to add a SegmentRefreshSemaphore class to wrap this logic.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Created a wrapper class.

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.

LGTM with minor comments

Comment on lines 239 to 242
} catch (Exception e) {
LOGGER.error("Caught exception while reloading segment: {} in table: {}", segmentName, tableNameWithType, e);
failedSegments.add(segmentName);
sampleException.set(e);
Copy link
Contributor

Choose a reason for hiding this comment

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

This catch is duplicated. We only need one try/catch/finally here

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The outside try-catch is to handle any InterruptedException while trying to acquire a permit (acquireSema throws an InterruptedException). If we put a finally in that block it'll try to release a permit in this case which would be wrong. So we need to another try-catch inside the outer try to execute the finally.

Copy link
Contributor

Choose a reason for hiding this comment

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

I see. In that case we should remove this catch block (keep the try/finally), and let the outside catch to handle the exception

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, that one is redundant. Addressed.

suddendust and others added 3 commits December 24, 2021 11:02
…reshSemaphore.java

Co-authored-by: Xiaotian (Jackie) Jiang <17555551+Jackie-Jiang@users.noreply.github.com>
@Jackie-Jiang Jackie-Jiang merged commit 99762cb into apache:master Dec 25, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

6 participants