lucene icon indicating copy to clipboard operation
lucene copied to clipboard

Make task executor non-final

Open Shibi-bala opened this issue 7 months ago • 12 comments

Description

The new task executor implementation from https://github.com/apache/lucene/pull/13472 means users can't exactly control the parallelism of the tasks being run. I think it is fair to allow task executor to be overridden in cases where users want the old or slightly different behavior.

One example of this change breaking consumers: if I had an executor service that I wanted to use to fairly control the execution of tasks across various queries, the calling thread would bypass this fair task handling completely.

Shibi-bala avatar Apr 18 '25 18:04 Shibi-bala

While I am not sure about changing to non-final, I am wondering if we should execute task on the current thread? Not sure if we save too much overhead and that makes code less readable.

Although, that was the case even before https://github.com/apache/lucene/pull/13861. So, I am wondering how did that change break the fair task handling?

jainankitk avatar Apr 22 '25 06:04 jainankitk

@jainankitk ah linked the wrong PR. It was https://github.com/apache/lucene/pull/13472 that made this change, and it seems like there were considerable speedups. Though it comes at the cost of making fair task handling very hard to control. Non-final was just the quickest solution, but happy to instead create an interface for this so consumers can implement their own task executor

Shibi-bala avatar Apr 22 '25 15:04 Shibi-bala

This really only impacts Knn queries where the work is done in rewrite and indexSearcher.getTaskExecutor() is called from the knn query. For all other queries, the caller can control what leaves are provided in indexSearcher.search(...) to control the parallelism. The current workaround is the create a new Knn query that submits tasks into a custom "task executor" instead of using the one provided by indexSearcher

Shibi-bala avatar Apr 22 '25 18:04 Shibi-bala

IMO Lucene should own how queries execute concurrently instead of making it pluggable. So I'd rather not allow users to pass a custom TaskExecutor.

jpountz avatar Apr 24 '25 12:04 jpountz

@jpountz IMO Lucene should own how queries execute concurrently instead of making it pluggable then why allow an executor service to be passed in? Previously, lucene didn't have control of this because a user's executor service could do whatever it wanted with the tasks. Now, control over execution is split between the user's executor service and what lucene does on the main query thread.

Shibi-bala avatar Apr 24 '25 21:04 Shibi-bala

Having introduced TaskExecutor in the first place and worked quite a bit on inter-segment concurrency and other ways to parallelize execution, the general idea was to make concurrency as transparent as possible. If we did mean to make TaskExecutor pluggable, it would not be just a matter of making it non final, but we would rather need to figure out what users need that is not currently available, and how they would plug in their custom behaviour. Could we start from something along these lines? How would you like execution to be parallelized in your example?

javanna avatar May 01 '25 12:05 javanna

@javanna I think what users need (or expect) with a TaskExecutor is to be able to have the created tasks run on the ExecutorService provided by the user. In my example, I'd like to be able to have tasks submitted to the ExecutorService to be run in a "fair" way. Basically, implementing query fairness by grouping tasks by originating user and estimated task cost. So my ideal TaskExecutor ideally just submits tasks and then collects results. The current implementation also would run tasks outside the "fair" ExecutorService which is why I want to plug in my own task executor to give up the optimization of running tasks on the calling thread in exchange for complete control of when and the order tasks are run.

Pluggable TaskExecutor is one solution here. As @jainankitk mentioned Not sure if we save too much overhead and that makes code less readable.

Shibi-bala avatar May 01 '25 20:05 Shibi-bala

Thanks for clarifying @Shibi-bala .

I get a bit confused here with the mention of query fairness. What #13472 does is advance execution on the caller thread, that would otherwise be waiting for the executor service's threads to complete their work, which is wasteful. At the same time it decreases the need for forking which provides better performance.

I find it counter-intuitive that to provide more fairness we would go back to a model where the caller thread waits most of its time, and does a lot more forking than needed. Is there a way to implement the grouping of fairness you have in mind on top of the current TaskExecutor? What I am thinking is that despite it may execute on the caller thread, it does still execute tasks in a certain order.

javanna avatar May 09 '25 09:05 javanna

This PR has not had activity in the past 2 weeks, labeling it as stale. If the PR is waiting for review, notify the [email protected] list. Thank you for your contribution!

github-actions[bot] avatar May 24 '25 00:05 github-actions[bot]

@javanna Hey! I kinda disagree with: general idea was to make concurrency as transparent as possible since the caller thread is used alongside another exec service. I think trading off slightly better perf in exchange for less transparency from the user pov isn't correct. In the prev case, it was very easy to see how resources were used and control that.

Is there a way to implement the grouping of fairness you have in mind on top of the current TaskExecutor?

Not that I am aware of, which is why I suggested to make the task executor pluggable.

Shibi-bala avatar May 29 '25 19:05 Shibi-bala

@javanna Hey! I kinda disagree with: general idea was to make concurrency as transparent as possible since the caller thread is used alongside another exec service. I think trading off slightly better perf in exchange for less transparency from the user pov isn't correct. In the prev case, it was very easy to see how resources were used and control that.

Is there a way to implement the grouping of fairness you have in mind on top of the current TaskExecutor?

Not that I am aware of, which is why I suggested to make the task executor pluggable.

@Shibi-bala I'm wondering whether it is what you want if you put your searcher.search() itself directly into the same Executor, and wait for it to complete (although I do admit this might not be best way to achieve your goal):

ExecutorService executor = Executors.newFixedThreadPool(4);
IndexSeacher searcher = new IndexSearcher(indexReader, executor);
executor.submit(() -> searcher.search()).get();

Please let me know if I got anything wrong

HUSTERGS avatar May 30 '25 18:05 HUSTERGS

This PR has not had activity in the past 2 weeks, labeling it as stale. If the PR is waiting for review, notify the [email protected] list. Thank you for your contribution!

github-actions[bot] avatar Jun 14 '25 00:06 github-actions[bot]

This PR has not had activity in the past 2 weeks, labeling it as stale. If the PR is waiting for review, notify the [email protected] list. Thank you for your contribution!

github-actions[bot] avatar Oct 16 '25 00:10 github-actions[bot]