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

Simplify task executor for concurrent operations #12499

Merged
merged 4 commits into from
Aug 21, 2023

Conversation

javanna
Copy link
Contributor

@javanna javanna commented Aug 10, 2023

Based on the proposal in #12498, this PR removes the QueueSizeBasedExecutor (package private) in favour of simply offloading concurrent execution to the provided executor. In need of specific behaviour, it can all be included in the executor itself.

This removes an instanceof check that determines which type of executor wrapper is used, which means that some tasks may be executed on the caller thread depending on queue size, whenever a rejection happens, or always for the last slice. This behaviour is not configurable in any way, and is too rigid. Rather than making this pluggable, I propose to make Lucene less opinionated about concurrent tasks execution and require that users include their own execution strategy directly in the executor that they provide to the index searcher.

Relates to #12498

IndexSearcher supports parallelizing search across slices when an executor is available. Knn queries can also parallelize their rewrite across segments using the same executor. Potentially, other operations will be parallelized in the future using the same pattern.

Lucene currently has the notion of a TaskExecutor (previously named SliceExecutor, but because knn rewrite is parallelized across segments rather than slices, it was recently renamed to TaskExecutor) which is responsible for offloading tasks to the executor, wait for them all to complete and return the corresponding results.

IndexSearcher currently has an instanceof check of the provided executor, and if it's a ThreadPoolExecutor, which is likely, a QueueSizeBasedExecutor is used which tries to be smart about when to offload tasks to the executor vs when to execute them on the caller thread. This behaviour is not configurable in any way.

As part of exposing concurrent search in Elasticsearch, we have found that the current task executor is too opinionated, and either it needs to be customizable, or simplified to be less opinionated.

In a server-side search engine, the search itself is likely executed by a thread pool that has its own sizing, queueing mechanisms as well as rejection policy. When offloading requests to an executor for additional parallelism, it is important to be able to determine where the load is going to be and what type of workload it maps to. Ideally, the caller thread is merely coordinating and not doing any I/O nor CPU intensive work, that is instead all delegated to the separate worker threads. Having built-in rules for when to execute certain operations on the caller thread may cause more problems than it solves, as it is unpredictable and makes sizing of thread pools more difficult, because all of a sudden you end up with two thread pools that may execute I/O as well as CPU intensive operations.

My conclusion is that if flexibility is needed in terms of possibly executing on the caller thread, such behaviour can be included in the executor that is provided to the searcher (for example with an adaptive mechanism that conditionally executes directly instead of offloading based on queue size like QueueSizeBasedExecutor does), together with its saturation policy (as opposed to catching RejectedExecutionException and executing on the caller thread, which is potentially dangerous). Also, executing the last slice / task on the caller thread, as it's the one waiting for all the tasks to complete, is not necessarily addressing a real problem around under-utilization of a thread that is doing nothing. That wait is cheap, and it's possibly more important to divide the workload among the thread pools. That said, such behavior can also be included in the Executor itself and does not require additional extension points.

My proposal is that we remove the QueueSizeBasedExecutor entirely and we simply offload every task to the executor, unconditionally. It's up to the provided executor to determine what to do when execute is called. That is the pluggable behaviour. Lucene should not have strong opinions nor provide building blocks for how to execute concurrent tasks.

Relates to apache#12498
@javanna
Copy link
Contributor Author

javanna commented Aug 17, 2023

Thanks for looking @reta ! i plan on merging this soon, if anyone else would like to review, please go ahead.

@reta
Copy link
Member

reta commented Aug 17, 2023

@sohami fyi

@sohami
Copy link
Contributor

sohami commented Aug 17, 2023

@sohami fyi

Thanks @reta for tagging me.

@sohami
Copy link
Contributor

sohami commented Aug 17, 2023

@javanna For single slice case are we thinking to leave it as is or change that as well to offload to the provided executor ?

@javanna
Copy link
Contributor Author

javanna commented Aug 21, 2023

@sohami I will open a follow-up to offload single slices too.

@javanna javanna added this to the 9.8.0 milestone Aug 21, 2023
@javanna javanna merged commit bb62720 into apache:main Aug 21, 2023
4 checks passed
@javanna
Copy link
Contributor Author

javanna commented Aug 21, 2023

Thanks all for looking!

javanna added a commit that referenced this pull request Aug 21, 2023
This commit removes the QueueSizeBasedExecutor (package private) in favour of simply offloading concurrent execution to the provided executor. In need of specific behaviour, it can all be included in the executor itself.

This removes an instanceof check that determines which type of executor wrapper is used, which means that some tasks may be executed on the caller thread depending on queue size, whenever a rejection happens, or always for the last slice. This behaviour is not configurable in any way, and is too rigid. Rather than making this pluggable, I propose to make Lucene less opinionated about concurrent tasks execution and require that users include their own execution strategy directly in the executor that they provide to the index searcher.

Relates to #12498
@javanna
Copy link
Contributor Author

javanna commented Aug 21, 2023

@sohami here it is: #12515

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.

3 participants