Skip to content

Remove search worker pool #111099

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

Conversation

original-brownbear
Copy link
Member

No more need for this pool, now that Lucene can safely execute on the current pool.

No need for this pool now that we can safely execute things on the
search pool.
@original-brownbear original-brownbear added WIP :Search/Search Search-related issues that do not fall into other categories labels Jul 19, 2024
Copy link
Contributor

@DaveCTurner DaveCTurner left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unfortunately this counts as a breaking change because each pool comes with settings to configure it that users might have set in their elasticsearch.yml files.

@original-brownbear
Copy link
Member Author

Unfortunately this counts as a breaking change because each pool comes with settings to configure it that users might have set in their elasticsearch.yml files.

True, I was planning to just make those settings noops in the final version of this, we're taking control over how things execute back again, this shouldn't be user configurable separately.

@javanna javanna added :Search Foundations/Search Catch all for Search Foundations and removed :Search/Search Search-related issues that do not fall into other categories labels Jul 19, 2024
@@ -17,6 +18,7 @@

import java.io.IOException;

@LuceneTestCase.AwaitsFix(bugUrl = "todo: this is somewhat meaningless with without a worker pool")
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we even want to keep tests like these? We have some others that check that parallelization actually happens,
what does it even mean that the min docs per slice jumps to 50k if concurrent search is disabled? :)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this was a way to verify ESSingleNodeTestCase behaviour, see #97740 . It's to check that when concurrency is enabled in tests, we generate as many slices as possible.

throw runtimeException;
}
throw new RuntimeException(e.getCause());
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

great to see this hack removed :)

@@ -17,6 +18,7 @@

import java.io.IOException;

@LuceneTestCase.AwaitsFix(bugUrl = "todo: this is somewhat meaningless with without a worker pool")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same as above, this is verifying that we customize the number of slices while running tests, to make sure we exercise the concurrent codepath when possible.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right, hmmm then I can't do the short-circuit the way I did it :/ So I guess I could put in the effort to make this thing create a more "realistic" search context with a result type that gives us parallelisation here to make this work? Let me see what I can do :)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You mean because we would have a single slice whenever an executor is not set? But don't we unset the executor only if we see a queue? I am missing something I think.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I piggy backed on the logic for figuring out the slice count because that was looking at the Executor type to get the thread-count so that I had a spot that gave me the queue reference.
I also figured that it's a neat optimization to simple return a single slice when there's a queue and then go down the non-forking code path right away.
So rather than just unsetting the executor for when there's a queue, I went for unsetting the executor when there's a single slice only and setting a single slice when there's a queue. Inverting the logic a little which seemed to me optimises things a little as well? Why slice when we're not planning on parallelising in the first place? :)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why slice when we're not planning on parallelising in the first place? :)

We would not slice if an executor isn't provided anyways. I will look more closely.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I looked, your logic looks correct to me, but I don't understand why this test and the other one need to be skipped. We have a test only minimum docs per slice setting, that gets lowered to 1 in tests to ensure that we do get slices during tests and we leverage the concurrent codepath (it gets passed to the index searcher whenever an executor is also provided. That is different from the max number of slices that you now use to decide whether you provide the executor or not. What am I missing?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What am I missing?

When I create the searcher in the non-concurrent path here https://github.com/elastic/elasticsearch/pull/111099/files#diff-5b3b75a4c26494593431361aefadf3faf536c865658dcbfd59a9b446ddc57cb3R190 it makes it so that the return of the max slices getter on the searcher just returns -1. So either we need to adjust the test here in some form or the prod code :)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I understand a bit better, that calling the search constructor that does not take an executor, min docs slice (that differs from max slice) is set to -1, which is what gets checked in this test. I guess that the problem is the more dynamic behavior of the decision making, as well as the fact that there are times where we do have an executor but with 1 max slice we don't provide it to the searcher.

I can't yet entirely follow where this test causes failures: do we end up with 1 max slice when concurrent search is enabled and we should have 1 min docs per slice?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I can't yet entirely follow where this test causes failures: do we end up with 1 max slice when concurrent search is enabled and we should have 1 min docs per slice?

Both paths return -1 for the min docs per slice so both fail with:


java.lang.AssertionError: 
Expected :50000
Actual   :-1
<Click to see difference>


	at __randomizedtesting.SeedInfo.seed([C73E4499C846CCD8:536C15AC617DAB14]:0)
	at org.junit.Assert.fail(Assert.java:89)
	at org.junit.Assert.failNotEquals(Assert.java:835)
	at org.junit.Assert.assertEquals(Assert.java:647)
	at org.junit.Assert.assertEquals(Assert.java:633)
	at org.elasticsearch.search.internal.ConcurrentSearchTestPluginTests.testConcurrentSearch(ConcurrentSearchTestPluginTests.java:37)
	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.base/java.lang.reflect.Method.invoke(Method.java:569)
	at com.carrotsearch.randomizedtesting.RandomizedRunner.invoke(RandomizedRunner.java:1758)
	at com.carrotsearch.randomizedtesting.RandomizedRunner$8.evaluate(RandomizedRunner.java:946)
	at com.carrotsearch.randomizedtesting.RandomizedRunner$9.evaluate(RandomizedRunner.java:982)
	at com.carrotsearch.randomizedtesting.RandomizedRunner$10.evaluate(RandomizedRunner.java:996)

... either expecting 50k or 1, but they always see -1 because the type in the search context is set to NONE and we now take the non-executor constructor path in this scenario.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@javanna couldn't find a neat way of testing this all the way through the search context. But I think that might not matter? All we want to test is that the setting is taken up? (if you agree, that's how I made it work now :))

@javanna
Copy link
Member

javanna commented Jul 24, 2024

I had a look, the direction looks good, I think that we need to update docs, as they have a mention of the search workers thread pool. I wonder if we need to do more to prevent breaking users that may be e.g. customizing the pool size etc.

Additionally, I think that we need to do something to prevent sub-tasks from being rejected. Once a shard search request has been accepted, its subtasks should not be rejected. This was previously ensured by the unbounded queue that the search workers thread pool had.

@original-brownbear
Copy link
Member Author

original-brownbear commented Jul 24, 2024

Thanks for taking a look @javanna :)

This was previously ensured by the unbounded queue that the search workers thread pool had.

This is an easy fix I think. We shouldn't practically run into it with the fact that we do not fork when when there's a queue but obviously lets be on the safe side here in theory as well :) Lets just wrap the executor passed to the searcher to ignore and execute tasks outright instead of bubbling up the rejection? (we could also just ignore rejections period with how Lucene works now, but maybe not the best idea to hard bake that in)

I think that we need to update docs, as they have a mention of the search workers thread pool. I wonder if we need to do more to prevent breaking users that may be e.g. customizing the pool size etc.

On it :)

@original-brownbear
Copy link
Member Author

Alrighty @javanna I removed the documentation referencing the search worker pool, deprecated the remaining settings related to it, found one more test simplification (snapshot resiliency tests) that we could make now and found a (hopefully not too dumb 🤞) fix for those tests I had to mute :) Let me know what you think + thanks for the patience here!

@original-brownbear original-brownbear marked this pull request as ready for review July 30, 2024 07:31
@original-brownbear original-brownbear requested a review from a team as a code owner July 30, 2024 07:31
@elasticsearchmachine elasticsearchmachine added the Team:Search Foundations Meta label for the Search Foundations team in Elasticsearch label Jul 30, 2024
@original-brownbear original-brownbear deleted the remove-worker-lucene-snapshot branch August 20, 2024 21:57
javanna added a commit that referenced this pull request Aug 29, 2024
This is a leftover from #111099: now that we use a single thread pool, we don't require offloading, which was the only
reason to carry around the executor in the time series index searcher, given that the number of slices is always 1 and it
does not support concurrency. Furthermore, providing the executor introduces some risks of potential concurrency caused by
non-slice based operations that can't be disabled. This commit removes the executor constructor argument and effectively
disable any concurrency in TimeSeriesIndexSearcher.
javanna added a commit that referenced this pull request Aug 29, 2024
This is a leftover from #111099: now that we use a single thread pool, we don't require offloading, which was the only
reason to carry around the executor in the time series index searcher, given that the number of slices is always 1 and it does not support concurrency. Furthermore, providing the executor introduces some risks of potential concurrency caused by non-slice based operations that can't be disabled. This commit removes the executor constructor argument and effectively disable any concurrency in TimeSeriesIndexSearcher.
javanna pushed a commit that referenced this pull request Aug 29, 2024
No more need for this pool, now that Lucene can safely execute on the current pool.
javanna added a commit to javanna/elasticsearch that referenced this pull request Oct 10, 2024
With recent changes in Lucene around not forking execution when not necessary, we have
removed the search worker thread pool in elastic#111099. The worker thread pool had unlimited queue,
and the fear was that we couuld have much more queueing on the search thread pool if we use
it to parallelize execution across segments, because every shard 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, hence this commit removes the additional queue dependent conditional in order
to perform additional benchmarks without it.
javanna added a commit that referenced this pull request Oct 10, 2024
With recent changes in Lucene around not forking execution when not necessary, we have
removed the search worker thread pool in #111099. The worker thread pool had unlimited queue,
and the fear was that we couuld have much more queueing on the search thread pool if we use
it to parallelize execution across segments, because every shard 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, hence this commit removes the additional queue dependent conditional in order
to perform additional benchmarks without it.
javanna added a commit to javanna/elasticsearch that referenced this pull request Oct 16, 2024
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.
javanna added a commit to javanna/elasticsearch that referenced this pull request Oct 16, 2024
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.
javanna added a commit that referenced this pull request Oct 16, 2024
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.
javanna added a commit to javanna/elasticsearch that referenced this pull request Oct 16, 2024
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.
javanna added a commit to javanna/elasticsearch that referenced this pull request Oct 16, 2024
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.
elasticsearchmachine pushed a commit that referenced this pull request Oct 16, 2024
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.
elasticsearchmachine pushed a commit that referenced this pull request Oct 17, 2024
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]>
georgewallace pushed a commit to georgewallace/elasticsearch that referenced this pull request Oct 25, 2024
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.
javanna added a commit to javanna/elasticsearch that referenced this pull request Oct 25, 2024
Since we removed the search workers thread pool with elastic#111099, we execute many
more tasks in the search thread pool, given that each shard search request
parallelizes across slices or even segments (knn query rewrite. There are also
rare situations where segment level tasks may parallelize further
(e.g. createWeight), that cause the creation of many many tasks for a single
top-level request. These are rather small tasks that previously queued up in
the unbounded search workers queue. With recent improvements in Lucene,
these tasks queue up in the search queue, yet they get executed by the caller
thread while they are still in the queue, and remain in the queue as no-op
until they are pulled out of the queue. We have protection against rejections
based on turning off search concurrency when we have more than maxPoolSize
items in the queue, yet that is not enough if enough parallel requests see
an empty queue and manage to submit enough tasks to fill the queue at once.
That will cause rejections for top-level searches that should not be rejected.

This commit introduces wrapping for the executor to limit the number of tasks
that a single search instance can submit to the executor, to prevent the situation
where a single search submits way more tasks than threads available.
javanna added a commit to javanna/elasticsearch that referenced this pull request Oct 25, 2024
Since we removed the search workers thread pool with elastic#111099, we execute many
more tasks in the search thread pool, given that each shard search request
parallelizes across slices or even segments (knn query rewrite. There are also
rare situations where segment level tasks may parallelize further
(e.g. createWeight), that cause the creation of many many tasks for a single
top-level request. These are rather small tasks that previously queued up in
the unbounded search workers queue. With recent improvements in Lucene,
these tasks queue up in the search queue, yet they get executed by the caller
thread while they are still in the queue, and remain in the queue as no-op
until they are pulled out of the queue. We have protection against rejections
based on turning off search concurrency when we have more than maxPoolSize
items in the queue, yet that is not enough if enough parallel requests see
an empty queue and manage to submit enough tasks to fill the queue at once.
That will cause rejections for top-level searches that should not be rejected.

This commit introduces wrapping for the executor to limit the number of tasks
that a single search instance can submit to the executor, to prevent the situation
where a single search submits way more tasks than threads available.
javanna added a commit to javanna/elasticsearch that referenced this pull request Oct 30, 2024
Since we removed the search workers thread pool with elastic#111099, we execute many
more tasks in the search thread pool, given that each shard search request
parallelizes across slices or even segments (knn query rewrite. There are also
rare situations where segment level tasks may parallelize further
(e.g. createWeight), that cause the creation of many many tasks for a single
top-level request. These are rather small tasks that previously queued up in
the unbounded search workers queue. With recent improvements in Lucene,
these tasks queue up in the search queue, yet they get executed by the caller
thread while they are still in the queue, and remain in the queue as no-op
until they are pulled out of the queue. We have protection against rejections
based on turning off search concurrency when we have more than maxPoolSize
items in the queue, yet that is not enough if enough parallel requests see
an empty queue and manage to submit enough tasks to fill the queue at once.
That will cause rejections for top-level searches that should not be rejected.

This commit introduces wrapping for the executor to limit the number of tasks
that a single search instance can submit to the executor, to prevent the situation
where a single search submits way more tasks than threads available.
javanna added a commit that referenced this pull request Oct 30, 2024
)

Since we removed the search workers thread pool with #111099, we execute many
more tasks in the search thread pool, given that each shard search request
parallelizes across slices or even segments (knn query rewrite. There are also
rare situations where segment level tasks may parallelize further
(e.g. createWeight), that cause the creation of many many tasks for a single
top-level request. These are rather small tasks that previously queued up in
the unbounded search workers queue. With recent improvements in Lucene,
these tasks queue up in the search queue, yet they get executed by the caller
thread while they are still in the queue, and remain in the queue as no-op
until they are pulled out of the queue. We have protection against rejections
based on turning off search concurrency when we have more than maxPoolSize
items in the queue, yet that is not enough if enough parallel requests see
an empty queue and manage to submit enough tasks to fill the queue at once.
That will cause rejections for top-level searches that should not be rejected.

This commit introduces wrapping for the executor to limit the number of tasks
that a single search instance can submit to the executor, to prevent the situation
where a single search submits way more tasks than threads available.

Co-authored-by: Adrien Grand <[email protected]>
javanna added a commit to javanna/elasticsearch that referenced this pull request Oct 30, 2024
…tic#115932)

Since we removed the search workers thread pool with elastic#111099, we execute many
more tasks in the search thread pool, given that each shard search request
parallelizes across slices or even segments (knn query rewrite. There are also
rare situations where segment level tasks may parallelize further
(e.g. createWeight), that cause the creation of many many tasks for a single
top-level request. These are rather small tasks that previously queued up in
the unbounded search workers queue. With recent improvements in Lucene,
these tasks queue up in the search queue, yet they get executed by the caller
thread while they are still in the queue, and remain in the queue as no-op
until they are pulled out of the queue. We have protection against rejections
based on turning off search concurrency when we have more than maxPoolSize
items in the queue, yet that is not enough if enough parallel requests see
an empty queue and manage to submit enough tasks to fill the queue at once.
That will cause rejections for top-level searches that should not be rejected.

This commit introduces wrapping for the executor to limit the number of tasks
that a single search instance can submit to the executor, to prevent the situation
where a single search submits way more tasks than threads available.

Co-authored-by: Adrien Grand <[email protected]>
elasticsearchmachine pushed a commit that referenced this pull request Oct 30, 2024
) (#115981)

Since we removed the search workers thread pool with #111099, we execute many
more tasks in the search thread pool, given that each shard search request
parallelizes across slices or even segments (knn query rewrite. There are also
rare situations where segment level tasks may parallelize further
(e.g. createWeight), that cause the creation of many many tasks for a single
top-level request. These are rather small tasks that previously queued up in
the unbounded search workers queue. With recent improvements in Lucene,
these tasks queue up in the search queue, yet they get executed by the caller
thread while they are still in the queue, and remain in the queue as no-op
until they are pulled out of the queue. We have protection against rejections
based on turning off search concurrency when we have more than maxPoolSize
items in the queue, yet that is not enough if enough parallel requests see
an empty queue and manage to submit enough tasks to fill the queue at once.
That will cause rejections for top-level searches that should not be rejected.

This commit introduces wrapping for the executor to limit the number of tasks
that a single search instance can submit to the executor, to prevent the situation
where a single search submits way more tasks than threads available.

Co-authored-by: Adrien Grand <[email protected]>
jfreden pushed a commit to jfreden/elasticsearch that referenced this pull request Nov 4, 2024
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.
@original-brownbear original-brownbear restored the remove-worker-lucene-snapshot branch November 30, 2024 10:08
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
>deprecation :Search Foundations/Search Catch all for Search Foundations Team:Search Foundations Meta label for the Search Foundations team in Elasticsearch v8.16.0 v9.0.0
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants