-
Notifications
You must be signed in to change notification settings - Fork 24.9k
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
Introduce executor for concurrent search #98204
Introduce executor for concurrent search #98204
Conversation
Elasticsearch has historically performed search sequentially across the segments. Lucene supports parallelizing search across segments when collecting hits (via collector managers) as well as when rewriting certain queries (e.g. knn queries). Elasticsearch is now ready to support concurrency within a single shard too. Search is already performed using collector managers, and the last piece that is missing is providing an executor to the index searcher so that it can offload the concurrent computation to it. This commit introduces a secondary executor, used exclusively to execute the concurrent bits of search. The search threads are still the ones that coordinate the search (where the caller search will originate from), but the actual work will be offloaded to the newly introduced executor. We are offloading not only parallel execution but also sequential execution, to make the workload more predictable, as it would be surprising to have bits of search executed in either of the two thread pools. Also, that would introduce the possibility to suddenly run a higher amount of heavy operations overall (some in the caller thread and some in the separate threads), which could overload the system as well as make sizing of thread pools more difficult. Note that fetch, together with other actions, is still executed in the search thread pool. This commit does not make the search thread pool merely a coordinating only thread pool, It does so only for what concerns the IndexSearcher#search operation itself, which is though a big portion of the different phases of search API execution. Given that the searcher blocks waiting for all tasks to be completed, we take a simple approach of introducing a bounded executor, which blocks whenever there are no threads to directly execute tasks. This simplifies handling of thread pool queue and rejections. In fact, we can guarantee that the secondary thread pool won't reject, and delegate queueing entirely to the search thread pool which is the entry point for every search operation anyway. The principle behind this is that if you got a slot in the search thread pool, you should be able to complete your search. As part of this commit we are also introducing the ability to cancel tasks that have not started yet, so that if any task throws an exception, other tasks are prevented from starting needless computation. This commit also enables concurrenct search execution in the DFS phase, which is going to improve resource usage as well as performance of knn queries which benefit from both concurrent rewrite and collection. We will enable concurrent execution for the query phase in a subsequent commit. Relates to elastic#80693 Relates to elastic#90700
Pinging @elastic/es-search (Team:Search) |
Hi @javanna, I've created a changelog YAML for you. |
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 am by no means an expert here and my review doesn't count for much, I had some minor things
try { | ||
semaphore.acquire(); | ||
} catch (InterruptedException e) { | ||
throw new ThreadInterruptedException(e); |
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.
If the thread is interrupted, shouldn't we bubble it up via Thread.currentThread().interrupt();
?
server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java
Outdated
Show resolved
Hide resolved
server/src/test/java/org/elasticsearch/search/internal/ContextIndexSearcherTests.java
Show resolved
Hide resolved
server/src/main/java/org/elasticsearch/search/SearchService.java
Outdated
Show resolved
Hide resolved
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.
Started on this and left a few initial comments.
server/src/main/java/org/elasticsearch/common/util/concurrent/BoundedExecutor.java
Outdated
Show resolved
Hide resolved
server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java
Outdated
Show resolved
Hide resolved
server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java
Outdated
Show resolved
Hide resolved
server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java
Outdated
Show resolved
Hide resolved
Co-authored-by: Henning Andersen <[email protected]>
@@ -502,7 +502,7 @@ public void apply(Settings value, Settings current, Settings previous) { | |||
ResourceWatcherService.RELOAD_INTERVAL_LOW, | |||
SearchModule.INDICES_MAX_CLAUSE_COUNT_SETTING, | |||
SearchModule.INDICES_MAX_NESTED_DEPTH_SETTING, | |||
SearchModule.SEARCH_CONCURRENCY_ENABLED, | |||
SearchService.SEARCH_WORKER_THREADS_ENABLED, |
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 have renamed the escape hatch to disable concurrency to align it with the name of the new thread pool, as it effectively affects whether the new thread pool is enabled or not.
server/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java
Outdated
Show resolved
Hide resolved
Setting.Property.NodeScope, | ||
Setting.Property.Dynamic | ||
); | ||
|
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 moved this to SearchService together with the other existing search settings (including minimum docs per slice which affects search concurrency)
@henningandersen @DaveCTurner I pushed an update to replace the bounded executor with a plain thread pool executor that relies on a synchronous queue. I added some questions for you two that I could use your help with. I am still working on the additional ContextIndexSearcherTests, that I just realized are not using the synchronous queue. Need to update those, work in progress. |
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 don't think a SynchronousQueue
has the right semantics here. AIUI it means that each .execute()
call will wait for the task to start executing on a worker thread, but I think we want to block the thread calling .execute()
until the task it submits is complete. Am I missing something there?
Blocking waiting for completion should be done by calling |
Sure, but can you help me understand why we need that? Is it just performance or are there semantic differences I'm missing? If it's performance, how much time does it save in practice? I'd prefer not to introduce yet another kind of queue (and another source of potential blocking to trip up the unwary) without a strong justification. |
This is already the case, it's how the |
We have introduced a search worker thread pool with elastic#98204 that is responsible for the heavy workloads as part of the query and dfs phase, no matter if it is parallelized across segments/slices or not. TSDB aggregations are still executed in the search thread pool and this commit moves their computation to the search worker thread pool, despite the corresponding search thread blocks and waits for such computation to be completed before returning.
We have introduced a search worker thread pool with #98204 that is responsible for the heavy workloads as part of the query and dfs phase, no matter if it is parallelized across segments/slices or not. TSDB aggregations are still executed in the search thread pool and this commit moves their computation to the search worker thread pool, despite the corresponding search thread blocks and waits for such computation to be completed before returning.
This commit enables concurrent search execution in the DFS phase, which is going to improve resource usage as well as performance of knn queries which benefit from both concurrent rewrite and collection. We will enable concurrent execution for the query phase in a subsequent commit. While this commit does not introduce parallelism for the query phase, it introduces offloading sequential computation to the newly introduced executor. This is true both for situations where a single slice needs to be searched, as well as scenarios where a specific request does not support concurrency (currently only DFS phase does regardless of the request). Sequential collection is not offloaded only if the request includes aggregations that don't support offloading: composite, nested and cardinality as their post collection method must be executed in the same thread as the collection or we'll trip a lucene assertion that verifies that doc_values are pulled and consumed from the same thread. ## Technical details This commit introduces a secondary executor, used exclusively to execute the concurrent bits of search. The search threads are still the ones that coordinate the search (where the caller search will originate from), but the actual work will be offloaded to the newly introduced executor. We are offloading not only parallel execution but also sequential execution, to make the workload more predictable, as it would be surprising to have bits of search executed in either of the two thread pools. Also, that would introduce the possibility to suddenly run a higher amount of heavy operations overall (some in the caller thread and some in the separate threads), which could overload the system as well as make sizing of thread pools more difficult. Note that fetch, together with other actions, is still executed in the search thread pool. This commit does not make the search thread pool merely a coordinating only thread pool, It does so only for what concerns the IndexSearcher#search operation itself, which is though a big portion of the different phases of search API execution. Given that the searcher blocks waiting for all tasks to be completed, we take a simple approach of introducing a thread pool executor that has the same size as the existing search thread pool but relies on an unbounded queue. This simplifies handling of thread pool queue and rejections. In fact, we'd like to guarantee that the secondary thread pool won't reject, and delegate queuing entirely to the search thread pool which is the entry point for every search operation anyway. The principle behind this is that if you got a slot in the search thread pool, you should be able to complete your search, and rather quickly. As part of this commit we are also introducing the ability to cancel tasks that have not started yet, so that if any task throws an exception, other tasks are prevented from starting needless computation. Relates to elastic#80693 Relates to elastic#90700
…98414) We have introduced a search worker thread pool with elastic#98204 that is responsible for the heavy workloads as part of the query and dfs phase, no matter if it is parallelized across segments/slices or not. TSDB aggregations are still executed in the search thread pool and this commit moves their computation to the search worker thread pool, despite the corresponding search thread blocks and waits for such computation to be completed before returning.
@javanna according to this PR's labels, I need to update the changelog YAML, but I can't because the PR is closed. Please either update the changelog yourself on the appropriate branch, or adjust the labels. Specifically:
|
With introduction of concurrent search across multiple segments elastic/elasticsearch#98204 there is a need to measure search across multiple segments before force merge. This PR adds this operation.
With introduction of concurrent search across multiple segments elastic/elasticsearch#98204 there is a need to measure search across multiple segments before force merge.
With introduction of concurrent search across multiple segments elastic/elasticsearch#98204 there is a need to measure search across multiple segments before force merge.
Enable concurrent search across segments for knn queries
Elasticsearch has historically performed search sequentially across the segments. Lucene supports parallelizing search across segments when collecting hits (via collector managers) as well as when rewriting certain queries (e.g. knn queries).
This commit also enables concurrent search execution in the DFS phase, which is going to improve resource usage as well as performance of knn queries which benefit from both concurrent rewrite and collection.
Enable offloading of sequential collection to search worker thread pool
We will enable concurrent execution for the query phase in a subsequent commit. While this commit does not introduce parallelism for the query phase, it introduces offloading sequential computation to the newly introduced executor. This is true both for situations where a single slice needs to be searched, as well as scenarios where a specific request does not support concurrency (currently only DFS phase does regardless of the request). Sequential collection is not offloaded only if the request includes aggregations that don't support offloading: composite, nested and cardinality as their post collection method must be executed in the same thread as the collection or we'll trip a lucene assertion that verifies that doc_values are pulled and consumed from the same thread.
Technical details
Elasticsearch is now ready to support concurrency within a single shard. Search is already performed using collector managers, and the last missing piece is providing an executor to the index searcher so that it can offload the concurrent computation to it.
This commit introduces a secondary executor, used exclusively to execute the concurrent bits of search. The search threads are still the ones that coordinate the search (where the caller search will originate from), but the actual work will be offloaded to the newly introduced executor.
We are offloading not only parallel execution but also sequential execution, to make the workload more predictable, as it would be surprising to have bits of search executed in either of the two thread pools. Also, that would introduce the possibility to suddenly run a higher amount of heavy operations overall (some in the caller thread and some in the separate threads), which could overload the system as well as make sizing of thread pools more difficult.
Note that fetch, together with other actions, is still executed in the search thread pool. This commit does not make the search thread pool merely a coordinating only thread pool, It does so only for what concerns the IndexSearcher#search operation itself, which is though a big portion of the different phases of search API execution.
Given that the searcher blocks waiting for all tasks to be completed, we take a simple approach of introducing a thread pool executor that has the same size as the existing search thread pool but relies on an unbounded queue. This simplifies handling of thread pool queue and rejections. In fact, we'd like to guarantee that the secondary thread pool won't reject, and delegate queuing entirely to the search thread pool which is the entry point for every search operation anyway. The principle behind this is that if you got a slot in the search thread pool, you should be able to complete your search, and rather quickly.
As part of this commit we are also introducing the ability to cancel tasks that have not started yet, so that if any task throws an exception, other tasks are prevented from starting needless computation.
Relates to #80693
Relates to #90700