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

[PARQUET-1055] Restrict size of ExecutorService when reading footers #418

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

Conversation

blacelle
Copy link

@blacelle blacelle commented Jul 5, 2017

No description provided.

Copy link
Contributor

@zivanfi zivanfi left a comment

Choose a reason for hiding this comment

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

Could you please create a JIRA describing the issue and reference it in the title of the PR? Thanks!

@@ -212,8 +212,20 @@ private static MetadataFilter filter(boolean skipRowGroups) {
}

private static <T> List<T> runAllInParallel(int parallelism, List<Callable<T>> toRun) throws ExecutionException {
LOG.info("Initiating action with parallelism: {}", parallelism);
ExecutorService threadPool = Executors.newFixedThreadPool(parallelism);
if (toRun.isEmpty()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I think this is a reasonable and safe optimization. If there is nothing to run, there is no need to create a threadpool.

ExecutorService threadPool = Executors.newFixedThreadPool(parallelism);
if (toRun.isEmpty()) {
return Collections.emptyList();
} else if (toRun.size() == 1) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't think this is a safe optimization. Running something in the current thread vs. running it in a separate thread could lead to different behavior. I believe that even if you caught all Throwable-s, it could still behave differently, so I think the single task case should not be handled differently.

Copy link
Author

Choose a reason for hiding this comment

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

Fine with me. 1 thread instead of 5 with the default config is still beneficial. Shall I update the PR with this change or you have more suggestions?

}
}
// Prevent instantiating a pool with many threads if not useful
int limitedParallelism = Math.min(parallelism, toRun.size());
Copy link
Contributor

Choose a reason for hiding this comment

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

I think this is a reasonable and safe optimization. There is no need to create a threadpool larger than the number of callables to run. However, I also think that it does not necessarily make sense to create that many either. For example, if there are 1000 callables, I would not create a separate thread for all of them. I think this number should be limited on the other boundary as well, i.e., it should not be more than Runtime.getRuntime().availableProcessors().

Copy link
Author

Choose a reason for hiding this comment

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

This is controlled by the parallelism parameter. If it is configured to parallelism=8 but the machine has 2 cores, then 8 threads may improve IO usage. I would not rely on Runtime.getRuntime().availableProcessors() to override parallelism

Copy link
Contributor

Choose a reason for hiding this comment

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

Sorry, my mistake. You're right.

@blacelle blacelle changed the title Restrict size of ExecutorService when reading footers [PARQUET-1055] Restrict size of ExecutorService when reading footers Jul 11, 2017
@blacelle
Copy link
Author

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.

2 participants