-
Notifications
You must be signed in to change notification settings - Fork 1k
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
TaskExecutor should not fork unnecessarily #13472
Conversation
When executing N tasks and waiting on the result of all of them, we should not fork one of them and just execute at least one of them outright. This saves at least one context switch, removes the need for any reentrancy protection, and makes full use of the supplied task executor if tasks fan out again.
Your suggestion is how it used to work before: https://github.com/apache/lucene/pull/12499/files#diff-e744fc99cb74627f02e30c1cbda56dede66d2ecdfd57db2ce869b9a9a43fa41cR49-R64. The context switching isn't great indeed, but executing some tasks on the current thread makes it hard to correctly reason about and configure the number of threads that should perform work. The idea behind this other PR was that you would have a worker executor that would do almost all the work, and a coordination executor that would be mostly coordinating work in the worker threadpool. I'm not sure if we're at a point where this coordination executor could run on virtual threads, but at least conceptually this is how I'm thinking of it. Something tangential that we touched a couple times but haven't implemented for now would consist of introducing an API on Also related, we are replacing some of the forking with the new support we introduced for I/O concurrency: https://github.com/apache/lucene/pull/13359/files#diff-ad7c504406afec8940592f1fda0062d3e5321cdc5693c24ec6f5cfb02f8dd90dL100-R114. |
But that's not what is happening practically right now? Whether I run N-1 tasks on the worker pool and one on the caller or N tasks on the worker and sleep on the caller thread, either way the coordinator thread is blocked and not coordinating anything in the meantime?
Not quite I think: The difference is that in my approach the coordinator thread keeps pulling stuff off of the queue in a loop, instead of just doing the last task. This means that the coordinating thread will not be "wasted" as much if the worker pool takes time to execute the tasks and can't do them all in parallel. |
That does make sense. But I wonder if this is something we should leave to the runtime/OS/... to figure out for us. It seems very desirable to limit the number of context switches when the API is synchronous so we can go through short tasks served from page cache with as little overhead as possible?
++ that'd be really nice and save a lot of overhead here. That said we could optimize this code fairly easily to move from waiting on "futures" to waiting on a single future :) I haven't benchmarked this much, but if we see non-trivial overhead for the wait loop due to frequent wakeup-sleep cycles as we go through all of the futures, we could just have a ref-count around a single future couldn't we?
❤️ |
Thanks for explaining I had not read your implementation carefully. I agree that we are doing less blocking than in the previous implementation of this, though we could still be blocking at times it seems? E.g. if you have two tasks and the first one takes more time? I understand what you are saying about reducing the overhead of forking, but I'm not too happy that it makes sizing thread pools more complicated in exchange? I need to think more about the trade-off. |
Right, that's a possible scenario, but unless we move to some kind of async API like the one you mentioned above, we'll always have blocking on the calling thread if there's a comparatively long running task running on one of the forked threads.
To me it seem like the opposite is true, this changes makes reasoning about the sizing much easier. I find it very complicated working out the relative sizes of worker pool and coordinator pool. With this change, I only have to size one pool and since blocking is rare I can probably ignore it in the calculation. So the size of the pool comes out to ~ |
I took some time to digest the suggested code changes and the discussions above. I get the sizing issues with using two thread pools (one executing If this change though makes us revisit the need for two pools, and allows users to provide the same executor that search already executes against, I think that would be a really good simplification. I need to think more about downsides, and expectations around sizing: we may need bigger queues, because a single search operation may create many more tasks than before? |
Right, an alternative would be to count in-progress searches at the top level and just make the queue unbounded? That would keep the behavior the same it is today and makes reasoning about the correct queue size simpler? Seems that's more of an ES than a Lucene concern though, Lucene should just make full use of the provided executor and that's that shouldn't it? |
Yes, I think so, but perhaps Lucene needs to provide general guidelines to users around what executor is suited and how it should be configured, what factors to take into account etc. |
Would it make sense to provide a reference implementation factory method that creates a properly-configured threadpool, maybe using all available cores with whatever appropriate policies? |
Lucene util benchmark results for this by running with one less thread for this branch vs main (credit to @jpountz and @javanna for the idea) to get an idea of the impact:
This is wikimediumall, 3 threads for main and 2 threads for this branch. Effectively no regressions but some considerable speedups.
to this branch
-> same number of instructions need to be executed pretty much, but they run in fewer cycles and encounter fewer cache misses. This is also seen in the profile of where the CPU time goes: main looks like this:
while this branch looks as follows:
-> a lot less time goes into |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I left some comments and questions, thanks a lot for opening this PR @original-brownbear !
The luceneutil results reported here are astounding. So astounding I'm not sure I believe them? I wonder if somehow we did not run with concurrency enabled on the main branch test ... or if there was some other testing artifact? Part of my thinking is that if this change was so impactful, then wouldn't we have seen a huge regression when moving from the prior situation (where we ran N-1 tasks in the threadpool and one task on the main thread) to the current situation? Hmm I do see your comment that this is different since the "main" thread is continuing to do more tasks. Still I'm really surprised at the impact. If I can get a moment I'll try to corroborate the test result. |
I had a similar reaction as you @msokolov. When it comes to lucene benchmarks, we only very recently started running those with concurrency enabled, so I think that previous changes to the concurrent code-path were not covered? In fact lucene-util did not provide the executor to the searcher until last week or so?
My general thinking is that there's a gain in using search concurrency in many cases, and some situations where it adds overhead or very little gain. At least when it comes to what we have observed in Elasticsearch benchmarks, we've so far mostly compared no concurrency with some concurrency, and not accurately observed differences between different attempts at running tasks concurrently: the root issue was the overhead of forking, and running no tasks in the caller thread vs one probably did not make a whole lot of difference. I think that the proposed solution is very different and that is why we are observing very different benchmark results. This change seems to take the concurrent code-path to a different level. With that, please do double check the bench results, more eyes on it can only help. |
@shubhamvishu @reta @sohami @mikemccand @zhaih I am pinging you folks because you have previously been involved in some of the search concurrency discussions and this PR will affect how search concurrency is exposed to users. Elasticsearch leverages inter-segment search concurrency in Lucene by providing an executor to the What do you all think? |
@msokolov they are astounding but in the opposite direction, in fact it's concurrency that's the problem mostly.
A large number of these items are actually showing extreme regressions from forking. Even this branch is like 50% behind no concurrency on some points. This is in fact how I got to opening this PR. When profiling ES benchmark runs I saw a bunch of sections where the overhead of forking for a given task was higher than the cost of just executing that same task right away. It's a little hard to quantitatively show this in a flame graph but the qualitative problem is here: And this is the same situation with my changes in Lucene: For weight creation, the forking overhead is still overwhelming but at least we save the future.get overhead from putting the calling thread to sleep and waking it up again. Only for longer running search tasks is the forking overhead "ok" I think. As I tried to show with the |
Thanks @jpountz, this change is quite clever, and I agree with @original-brownbear that it leads to better CPU utilization since the caller thread "joins" the executor pool in the effort (also confirmed by benchmarks). May be I am missing something, but the implementation basically introduces "double queuing": task executor has one and very likely the supplied On the general note, it resembles a lot the way |
You're right this is one of the remaining areas of contention that could be fixed for even better performance. Using a |
This change is definitely an improvement (over the existing implementation)
Eliminate - probably not, but reduce further - likely but it needs yet to be proven (as you rightly pointed out, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
Thanks @original-brownbear ! |
When an executor is provided to the IndexSearcher constructor, the searcher now executes tasks on the thread that invoked a search as well as its configured executor. Users should reduce the executor's thread-count by 1 to retain the previous level of parallelism. Moreover, it is now possible to start searches from the same executor that is configured in the IndexSearcher without risk of deadlocking. A separate executor for starting searches is no longer required. Previously, a separate executor was required to prevent deadlock, and all the tasks were offloaded to it unconditionally, wasting resources in some scenarios due to unnecessary forking, and the caller thread having to wait for all tasks to be completed anyways. it can now actively contribute to the execution as well.
When an executor is provided to the IndexSearcher constructor, the searcher now executes tasks on the thread that invoked a search as well as its configured executor. Users should reduce the executor's thread-count by 1 to retain the previous level of parallelism. Moreover, it is now possible to start searches from the same executor that is configured in the IndexSearcher without risk of deadlocking. A separate executor for starting searches is no longer required. Previously, a separate executor was required to prevent deadlock, and all the tasks were offloaded to it unconditionally, wasting resources in some scenarios due to unnecessary forking, and the caller thread having to wait for all tasks to be completed anyways. it can now actively contribute to the execution as well. Co-authored-by: Armin Braun <[email protected]>
Should we now update |
@jpountz yea makes sense to me. |
@original-brownbear Would you like to work on a PR? |
I just pushed an annotation for this change: mikemccand/luceneutil@a64ac17. Several queries got a bit faster with a low p-value: https://people.apache.org/~mikemccand/lucenebench/2024.07.05.14.34.52.html. |
Sure thing, on it! :) sorry could've done that right away, tired me just didn't realise it this morning . |
Even with the recent Lucene improvements in apache/lucene#13472 there is no need to go through all the Lucene machinery for a single slice here. The task executor allocates a bunch of objects and runs into some memory barriers that the JVM can't necessarily compile away. Lets save that overhead for the single slice case. Also this PR removes the pointless list of collectors that we were building.
With recent changes in Lucene 9.12 around not forking execution when not necessary (see apache/lucene#13472), we have removed the search worker thread pool in elastic#111099. The worker thread pool had unlimited queue, and we feared that we couuld have much more queueing on the search thread pool if we execute segment level searches on the same thread pool as the shard level searches, because every shard search would take up to a thread per slice when executing the query phase. We have then introduced an additional conditional to stop parallelizing when there is a queue. That is perhaps a bit extreme, as it's a decision made when creating the searcher, while a queue may no longer be there once the search is executing. This has caused some benchmarks regressions, given that having a queue may be a transient scenario, especially with short-lived segment searches being queued up. We may end up disabling inter-segment concurrency more aggressively than we would want, penalizing requests that do benefit from concurrency. At the same time, we do want to have some kind of protection against rejections of shard searches that would be caused by excessive slicing. When the queue is above a certain size, we can turn off the slicing and effectively disable inter-segment concurrency. With this commit we set that threshold to be the number of threads in the search pool.
With recent changes in Lucene 9.12 around not forking execution when not necessary (see apache/lucene#13472), we have removed the search worker thread pool in elastic#111099. The worker thread pool had unlimited queue, and we feared that we couuld have much more queueing on the search thread pool if we execute segment level searches on the same thread pool as the shard level searches, because every shard search would take up to a thread per slice when executing the query phase. We have then introduced an additional conditional to stop parallelizing when there is a queue. That is perhaps a bit extreme, as it's a decision made when creating the searcher, while a queue may no longer be there once the search is executing. This has caused some benchmarks regressions, given that having a queue may be a transient scenario, especially with short-lived segment searches being queued up. We may end up disabling inter-segment concurrency more aggressively than we would want, penalizing requests that do benefit from concurrency. At the same time, we do want to have some kind of protection against rejections of shard searches that would be caused by excessive slicing. When the queue is above a certain size, we can turn off the slicing and effectively disable inter-segment concurrency. With this commit we set that threshold to be the number of threads in the search pool.
With recent changes in Lucene 9.12 around not forking execution when not necessary (see apache/lucene#13472), we have removed the search worker thread pool in #111099. The worker thread pool had unlimited queue, and we feared that we couuld have much more queueing on the search thread pool if we execute segment level searches on the same thread pool as the shard level searches, because every shard search would take up to a thread per slice when executing the query phase. We have then introduced an additional conditional to stop parallelizing when there is a queue. That is perhaps a bit extreme, as it's a decision made when creating the searcher, while a queue may no longer be there once the search is executing. This has caused some benchmarks regressions, given that having a queue may be a transient scenario, especially with short-lived segment searches being queued up. We may end up disabling inter-segment concurrency more aggressively than we would want, penalizing requests that do benefit from concurrency. At the same time, we do want to have some kind of protection against rejections of shard searches that would be caused by excessive slicing. When the queue is above a certain size, we can turn off the slicing and effectively disable inter-segment concurrency. With this commit we set that threshold to be the number of threads in the search pool.
With recent changes in Lucene 9.12 around not forking execution when not necessary (see apache/lucene#13472), we have removed the search worker thread pool in elastic#111099. The worker thread pool had unlimited queue, and we feared that we couuld have much more queueing on the search thread pool if we execute segment level searches on the same thread pool as the shard level searches, because every shard search would take up to a thread per slice when executing the query phase. We have then introduced an additional conditional to stop parallelizing when there is a queue. That is perhaps a bit extreme, as it's a decision made when creating the searcher, while a queue may no longer be there once the search is executing. This has caused some benchmarks regressions, given that having a queue may be a transient scenario, especially with short-lived segment searches being queued up. We may end up disabling inter-segment concurrency more aggressively than we would want, penalizing requests that do benefit from concurrency. At the same time, we do want to have some kind of protection against rejections of shard searches that would be caused by excessive slicing. When the queue is above a certain size, we can turn off the slicing and effectively disable inter-segment concurrency. With this commit we set that threshold to be the number of threads in the search pool.
With recent changes in Lucene 9.12 around not forking execution when not necessary (see apache/lucene#13472), we have removed the search worker thread pool in elastic#111099. The worker thread pool had unlimited queue, and we feared that we couuld have much more queueing on the search thread pool if we execute segment level searches on the same thread pool as the shard level searches, because every shard search would take up to a thread per slice when executing the query phase. We have then introduced an additional conditional to stop parallelizing when there is a queue. That is perhaps a bit extreme, as it's a decision made when creating the searcher, while a queue may no longer be there once the search is executing. This has caused some benchmarks regressions, given that having a queue may be a transient scenario, especially with short-lived segment searches being queued up. We may end up disabling inter-segment concurrency more aggressively than we would want, penalizing requests that do benefit from concurrency. At the same time, we do want to have some kind of protection against rejections of shard searches that would be caused by excessive slicing. When the queue is above a certain size, we can turn off the slicing and effectively disable inter-segment concurrency. With this commit we set that threshold to be the number of threads in the search pool.
With recent changes in Lucene 9.12 around not forking execution when not necessary (see apache/lucene#13472), we have removed the search worker thread pool in #111099. The worker thread pool had unlimited queue, and we feared that we couuld have much more queueing on the search thread pool if we execute segment level searches on the same thread pool as the shard level searches, because every shard search would take up to a thread per slice when executing the query phase. We have then introduced an additional conditional to stop parallelizing when there is a queue. That is perhaps a bit extreme, as it's a decision made when creating the searcher, while a queue may no longer be there once the search is executing. This has caused some benchmarks regressions, given that having a queue may be a transient scenario, especially with short-lived segment searches being queued up. We may end up disabling inter-segment concurrency more aggressively than we would want, penalizing requests that do benefit from concurrency. At the same time, we do want to have some kind of protection against rejections of shard searches that would be caused by excessive slicing. When the queue is above a certain size, we can turn off the slicing and effectively disable inter-segment concurrency. With this commit we set that threshold to be the number of threads in the search pool.
With recent changes in Lucene 9.12 around not forking execution when not necessary (see apache/lucene#13472), we have removed the search worker thread pool in #111099. The worker thread pool had unlimited queue, and we feared that we couuld have much more queueing on the search thread pool if we execute segment level searches on the same thread pool as the shard level searches, because every shard search would take up to a thread per slice when executing the query phase. We have then introduced an additional conditional to stop parallelizing when there is a queue. That is perhaps a bit extreme, as it's a decision made when creating the searcher, while a queue may no longer be there once the search is executing. This has caused some benchmarks regressions, given that having a queue may be a transient scenario, especially with short-lived segment searches being queued up. We may end up disabling inter-segment concurrency more aggressively than we would want, penalizing requests that do benefit from concurrency. At the same time, we do want to have some kind of protection against rejections of shard searches that would be caused by excessive slicing. When the queue is above a certain size, we can turn off the slicing and effectively disable inter-segment concurrency. With this commit we set that threshold to be the number of threads in the search pool. Co-authored-by: Elastic Machine <[email protected]>
With recent changes in Lucene 9.12 around not forking execution when not necessary (see apache/lucene#13472), we have removed the search worker thread pool in elastic#111099. The worker thread pool had unlimited queue, and we feared that we couuld have much more queueing on the search thread pool if we execute segment level searches on the same thread pool as the shard level searches, because every shard search would take up to a thread per slice when executing the query phase. We have then introduced an additional conditional to stop parallelizing when there is a queue. That is perhaps a bit extreme, as it's a decision made when creating the searcher, while a queue may no longer be there once the search is executing. This has caused some benchmarks regressions, given that having a queue may be a transient scenario, especially with short-lived segment searches being queued up. We may end up disabling inter-segment concurrency more aggressively than we would want, penalizing requests that do benefit from concurrency. At the same time, we do want to have some kind of protection against rejections of shard searches that would be caused by excessive slicing. When the queue is above a certain size, we can turn off the slicing and effectively disable inter-segment concurrency. With this commit we set that threshold to be the number of threads in the search pool.
With recent changes in Lucene 9.12 around not forking execution when not necessary (see apache/lucene#13472), we have removed the search worker thread pool in elastic#111099. The worker thread pool had unlimited queue, and we feared that we couuld have much more queueing on the search thread pool if we execute segment level searches on the same thread pool as the shard level searches, because every shard search would take up to a thread per slice when executing the query phase. We have then introduced an additional conditional to stop parallelizing when there is a queue. That is perhaps a bit extreme, as it's a decision made when creating the searcher, while a queue may no longer be there once the search is executing. This has caused some benchmarks regressions, given that having a queue may be a transient scenario, especially with short-lived segment searches being queued up. We may end up disabling inter-segment concurrency more aggressively than we would want, penalizing requests that do benefit from concurrency. At the same time, we do want to have some kind of protection against rejections of shard searches that would be caused by excessive slicing. When the queue is above a certain size, we can turn off the slicing and effectively disable inter-segment concurrency. With this commit we set that threshold to be the number of threads in the search pool.
When executing N tasks and waiting on the result of all of them, we should only fork N - 1 times and execute one task on the calling thread that is getting blocked anyway. This saves at least one context switch, removes the need for any reentrancy protection, and makes better use of available CPU resources.