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 #12498

Closed
javanna opened this issue Aug 10, 2023 · 6 comments
Closed

Simplify task executor for concurrent operations #12498

javanna opened this issue Aug 10, 2023 · 6 comments
Labels
discussion Discussion
Milestone

Comments

@javanna
Copy link
Contributor

javanna commented Aug 10, 2023

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.

Additionally, I think that we should unconditionally offload execution to the executor when available, even when we have a single slice. It may seem counter intuitive but it's again to be able to determine what type of workload each thread pool performs.

Relates to #12438

@javanna javanna added the discussion Discussion label Aug 10, 2023
@javanna
Copy link
Contributor Author

javanna commented Aug 10, 2023

@jpountz @mikemccand pinging you two because you are likely to have thoughts on this.

javanna added a commit to javanna/lucene that referenced this issue Aug 10, 2023
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
@jpountz
Copy link
Contributor

jpountz commented Aug 11, 2023

It makes sense to me to push the responsibility of figuring out how to execute tasks to the executor. Also pinging @reta.

@reta
Copy link
Member

reta commented Aug 11, 2023

It makes sense to me to push the responsibility of figuring out how to execute tasks to the executor. Also pinging @reta.

Thanks @jpountz , I second that

Additionally, I think that we should unconditionally offload execution to the executor when available, even when we have a single slice. It may seem counter intuitive but it's again to be able to determine what type of workload each thread pool performs.

That's is one of the difficulties we are dealing as well, specifically the exception branching logic has to account for wrapped / unwrapped exceptions.

@javanna
Copy link
Contributor Author

javanna commented Aug 17, 2023

Thanks for the feedback everyone, there's a PR up as a first step (does not include single slice offloading yet): #12499 . Reviews are welcome.

javanna added a commit that referenced this issue 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 added a commit that referenced this issue 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 added a commit to javanna/lucene that referenced this issue Aug 21, 2023
When an executor is set to the IndexSearcher, we shoudl try and offload
most of the computation to such executor. Ideally, the caller thread
would only do light coordination work, and the executor is responsible
for the heavier workload. If we don't offload sequential execution to
the executor, it becomes very difficult to make any distinction about
the type of workload performed on the two sides.

Closes apache#12498
javanna added a commit to javanna/lucene that referenced this issue Aug 21, 2023
When an executor is set to the IndexSearcher, we should try and offload
most of the computation to such executor. Ideally, the caller thread
would only do light coordination work, and the executor is responsible
for the heavier workload. If we don't offload sequential execution to
the executor, it becomes very difficult to make any distinction about
the type of workload performed on the two sides.

Closes apache#12498
@Jeevananthan-23
Copy link

Hi @javanna I opened #12531 for async task processing, It could be great to use VirtualThreads(JDK21) for IndexSearch which able concurrent tasks execution similar to Executor to run in ThreadPool Carrier Thread rather it runs in lightweight VirtualThreads.

@uschindler
Copy link
Contributor

uschindler commented Sep 4, 2023

You can use virtual threads out of box for IndexSearcher. Just pass a suitable executor: Executors.newVirtualThreadPerTaskExecutor()

@javanna javanna closed this as completed in da89415 Sep 5, 2023
javanna added a commit that referenced this issue Sep 5, 2023
When an executor is set to the IndexSearcher, we should try and offload
most of the computation to such executor. Ideally, the caller thread
would only do light coordination work, and the executor is responsible
for the heavier workload. If we don't offload sequential execution to
the executor, it becomes very difficult to make any distinction about
the type of workload performed on the two sides.

Closes #12498
javanna added a commit that referenced this issue Sep 5, 2023
javanna added a commit that referenced this issue Sep 5, 2023
@zhaih zhaih added this to the 9.8.0 milestone Sep 20, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
discussion Discussion
Projects
None yet
Development

No branches or pull requests

6 participants