Skip to content

Commit 6bf2c71

Browse files
authored
Limit concurrent shards per node for ESQL (#104832)
Today, we allow ESQL to execute against an unlimited number of shards concurrently on each node. This can lead to cases where we open and hold too many shards, equivalent to opening too many file descriptors or using too much memory for FieldInfos in ValuesSourceReaderOperator. This change limits the number of concurrent shards to 10 per node. This number was chosen based on the _search API, which limits it to 5. Besides the primary reason stated above, this change has other implications: We might execute fewer shards for queries with LIMIT only, leading to scenarios where we execute only some high-priority shards then stop. For now, we don't have a partial reduce at the node level, but if we introduce one in the future, it might not be as efficient as executing all shards at the same time. There are pauses between batches because batches are executed sequentially one by one. However, I believe the performance of queries executing against many shards (after can_match) is less important than resiliency. Closes #103666 Backport of #104832
1 parent b3bca4b commit 6bf2c71

File tree

10 files changed

+212
-36
lines changed

10 files changed

+212
-36
lines changed

docs/changelog/104832.yaml

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,6 @@
1+
pr: 104832
2+
summary: Limit concurrent shards per node for ESQL
3+
area: ES|QL
4+
type: bug
5+
issues:
6+
- 103666

server/src/main/java/org/elasticsearch/search/SearchService.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1060,7 +1060,7 @@ protected SearchContext createContext(
10601060
return context;
10611061
}
10621062

1063-
public DefaultSearchContext createSearchContext(ShardSearchRequest request, TimeValue timeout) throws IOException {
1063+
public SearchContext createSearchContext(ShardSearchRequest request, TimeValue timeout) throws IOException {
10641064
final IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
10651065
final IndexShard indexShard = indexService.getShard(request.shardId().getId());
10661066
final Engine.SearcherSupplier reader = indexShard.acquireSearcherSupplier();

server/src/test/java/org/elasticsearch/search/SearchServiceTests.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1264,7 +1264,7 @@ public void testCreateSearchContext() throws IOException {
12641264
nowInMillis,
12651265
clusterAlias
12661266
);
1267-
try (DefaultSearchContext searchContext = service.createSearchContext(request, new TimeValue(System.currentTimeMillis()))) {
1267+
try (SearchContext searchContext = service.createSearchContext(request, new TimeValue(System.currentTimeMillis()))) {
12681268
SearchShardTarget searchShardTarget = searchContext.shardTarget();
12691269
SearchExecutionContext searchExecutionContext = searchContext.getSearchExecutionContext();
12701270
String expectedIndexName = clusterAlias == null ? index : clusterAlias + ":" + index;

test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -11,6 +11,7 @@
1111
import org.elasticsearch.action.search.SearchShardTask;
1212
import org.elasticsearch.cluster.service.ClusterService;
1313
import org.elasticsearch.common.util.BigArrays;
14+
import org.elasticsearch.core.TimeValue;
1415
import org.elasticsearch.indices.ExecutorSelector;
1516
import org.elasticsearch.indices.IndicesService;
1617
import org.elasticsearch.indices.breaker.CircuitBreakerService;
@@ -41,6 +42,7 @@ public static class TestPlugin extends Plugin {}
4142
private static final Map<ReaderContext, Throwable> ACTIVE_SEARCH_CONTEXTS = new ConcurrentHashMap<>();
4243

4344
private Consumer<ReaderContext> onPutContext = context -> {};
45+
private Consumer<ReaderContext> onRemoveContext = context -> {};
4446

4547
private Consumer<SearchContext> onCreateSearchContext = context -> {};
4648

@@ -110,6 +112,7 @@ protected void putReaderContext(ReaderContext context) {
110112
protected ReaderContext removeReaderContext(long id) {
111113
final ReaderContext removed = super.removeReaderContext(id);
112114
if (removed != null) {
115+
onRemoveContext.accept(removed);
113116
removeActiveContext(removed);
114117
}
115118
return removed;
@@ -119,6 +122,10 @@ public void setOnPutContext(Consumer<ReaderContext> onPutContext) {
119122
this.onPutContext = onPutContext;
120123
}
121124

125+
public void setOnRemoveContext(Consumer<ReaderContext> onRemoveContext) {
126+
this.onRemoveContext = onRemoveContext;
127+
}
128+
122129
public void setOnCreateSearchContext(Consumer<SearchContext> onCreateSearchContext) {
123130
this.onCreateSearchContext = onCreateSearchContext;
124131
}
@@ -141,6 +148,14 @@ protected SearchContext createContext(
141148
return searchContext;
142149
}
143150

151+
@Override
152+
public SearchContext createSearchContext(ShardSearchRequest request, TimeValue timeout) throws IOException {
153+
SearchContext searchContext = super.createSearchContext(request, timeout);
154+
onPutContext.accept(searchContext.readerContext());
155+
searchContext.addReleasable(() -> onRemoveContext.accept(searchContext.readerContext()));
156+
return searchContext;
157+
}
158+
144159
public void setOnCheckCancelled(Function<SearchShardTask, SearchShardTask> onCheckCancelled) {
145160
this.onCheckCancelled = onCheckCancelled;
146161
}

x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSinkHandler.java

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -108,7 +108,10 @@ public void addCompletionListener(ActionListener<Void> listener) {
108108
completionFuture.addListener(listener);
109109
}
110110

111-
boolean isFinished() {
111+
/**
112+
* Returns true if an exchange is finished
113+
*/
114+
public boolean isFinished() {
112115
return completionFuture.isDone();
113116
}
114117

x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractEsqlIntegTestCase.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -164,6 +164,9 @@ protected static QueryPragmas randomPragmas() {
164164
};
165165
settings.put("page_size", pageSize);
166166
}
167+
if (randomBoolean()) {
168+
settings.put("max_concurrent_shards_per_node", randomIntBetween(1, 10));
169+
}
167170
}
168171
return new QueryPragmas(settings.build());
169172
}

x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/ManyShardsIT.java

Lines changed: 74 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -13,19 +13,36 @@
1313
import org.elasticsearch.action.support.WriteRequest;
1414
import org.elasticsearch.cluster.metadata.IndexMetadata;
1515
import org.elasticsearch.common.settings.Settings;
16+
import org.elasticsearch.plugins.Plugin;
17+
import org.elasticsearch.search.MockSearchService;
18+
import org.elasticsearch.search.SearchService;
1619
import org.elasticsearch.xpack.esql.plugin.QueryPragmas;
20+
import org.hamcrest.Matchers;
21+
import org.junit.Before;
1722

23+
import java.util.ArrayList;
24+
import java.util.Collection;
25+
import java.util.List;
1826
import java.util.Map;
1927
import java.util.concurrent.CountDownLatch;
2028
import java.util.concurrent.TimeUnit;
29+
import java.util.concurrent.atomic.AtomicInteger;
2130

2231
/**
2332
* Make sures that we can run many concurrent requests with large number of shards with any data_partitioning.
2433
*/
2534
@LuceneTestCase.SuppressFileSystems(value = "HandleLimitFS")
2635
public class ManyShardsIT extends AbstractEsqlIntegTestCase {
2736

28-
public void testConcurrentQueries() throws Exception {
37+
@Override
38+
protected Collection<Class<? extends Plugin>> getMockPlugins() {
39+
var plugins = new ArrayList<>(super.getMockPlugins());
40+
plugins.add(MockSearchService.TestPlugin.class);
41+
return plugins;
42+
}
43+
44+
@Before
45+
public void setupIndices() {
2946
int numIndices = between(10, 20);
3047
for (int i = 0; i < numIndices; i++) {
3148
String index = "test-" + i;
@@ -49,6 +66,9 @@ public void testConcurrentQueries() throws Exception {
4966
}
5067
bulk.get();
5168
}
69+
}
70+
71+
public void testConcurrentQueries() throws Exception {
5272
int numQueries = between(10, 20);
5373
Thread[] threads = new Thread[numQueries];
5474
CountDownLatch latch = new CountDownLatch(1);
@@ -76,4 +96,57 @@ public void testConcurrentQueries() throws Exception {
7696
thread.join();
7797
}
7898
}
99+
100+
static class SearchContextCounter {
101+
private final int maxAllowed;
102+
private final AtomicInteger current = new AtomicInteger();
103+
104+
SearchContextCounter(int maxAllowed) {
105+
this.maxAllowed = maxAllowed;
106+
}
107+
108+
void onNewContext() {
109+
int total = current.incrementAndGet();
110+
assertThat("opening more shards than the limit", total, Matchers.lessThanOrEqualTo(maxAllowed));
111+
}
112+
113+
void onContextReleased() {
114+
int total = current.decrementAndGet();
115+
assertThat(total, Matchers.greaterThanOrEqualTo(0));
116+
}
117+
}
118+
119+
public void testLimitConcurrentShards() {
120+
Iterable<SearchService> searchServices = internalCluster().getInstances(SearchService.class);
121+
try {
122+
var queries = List.of(
123+
"from test-* | stats count(user) by tags",
124+
"from test-* | stats count(user) by tags | LIMIT 0",
125+
"from test-* | stats count(user) by tags | LIMIT 1",
126+
"from test-* | stats count(user) by tags | LIMIT 1000",
127+
"from test-* | LIMIT 0",
128+
"from test-* | LIMIT 1",
129+
"from test-* | LIMIT 1000",
130+
"from test-* | SORT tags | LIMIT 0",
131+
"from test-* | SORT tags | LIMIT 1",
132+
"from test-* | SORT tags | LIMIT 1000"
133+
);
134+
for (String q : queries) {
135+
QueryPragmas pragmas = randomPragmas();
136+
for (SearchService searchService : searchServices) {
137+
SearchContextCounter counter = new SearchContextCounter(pragmas.maxConcurrentShardsPerNode());
138+
var mockSearchService = (MockSearchService) searchService;
139+
mockSearchService.setOnPutContext(r -> counter.onNewContext());
140+
mockSearchService.setOnRemoveContext(r -> counter.onContextReleased());
141+
}
142+
run(q, pragmas).close();
143+
}
144+
} finally {
145+
for (SearchService searchService : searchServices) {
146+
var mockSearchService = (MockSearchService) searchService;
147+
mockSearchService.setOnPutContext(r -> {});
148+
mockSearchService.setOnRemoveContext(r -> {});
149+
}
150+
}
151+
}
79152
}

x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/WarningsIT.java

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@
99

1010
import org.elasticsearch.action.ActionListener;
1111
import org.elasticsearch.action.support.PlainActionFuture;
12+
import org.elasticsearch.cluster.metadata.IndexMetadata;
1213
import org.elasticsearch.cluster.node.DiscoveryNode;
1314
import org.elasticsearch.common.settings.Settings;
1415
import org.elasticsearch.test.junit.annotations.TestLogging;
@@ -40,7 +41,11 @@ public void testCollectWarnings() {
4041
client().admin()
4142
.indices()
4243
.prepareCreate("index-1")
43-
.setSettings(Settings.builder().put("index.routing.allocation.require._name", node1))
44+
.setSettings(
45+
Settings.builder()
46+
.put("index.routing.allocation.require._name", node1)
47+
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5))
48+
)
4449
.setMapping("host", "type=keyword")
4550
);
4651
for (int i = 0; i < numDocs1; i++) {
@@ -51,7 +56,11 @@ public void testCollectWarnings() {
5156
client().admin()
5257
.indices()
5358
.prepareCreate("index-2")
54-
.setSettings(Settings.builder().put("index.routing.allocation.require._name", node2))
59+
.setSettings(
60+
Settings.builder()
61+
.put("index.routing.allocation.require._name", node2)
62+
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5))
63+
)
5564
.setMapping("host", "type=keyword")
5665
);
5766
for (int i = 0; i < numDocs2; i++) {

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java

Lines changed: 87 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@
1616
import org.elasticsearch.action.search.SearchShardsRequest;
1717
import org.elasticsearch.action.search.SearchShardsResponse;
1818
import org.elasticsearch.action.search.TransportSearchShardsAction;
19+
import org.elasticsearch.action.support.ChannelActionListener;
1920
import org.elasticsearch.action.support.ContextPreservingActionListener;
2021
import org.elasticsearch.action.support.RefCountingListener;
2122
import org.elasticsearch.action.support.RefCountingRunnable;
@@ -26,7 +27,6 @@
2627
import org.elasticsearch.common.io.stream.StreamOutput;
2728
import org.elasticsearch.common.util.BigArrays;
2829
import org.elasticsearch.common.util.concurrent.ThreadContext;
29-
import org.elasticsearch.compute.OwningChannelActionListener;
3030
import org.elasticsearch.compute.data.BlockFactory;
3131
import org.elasticsearch.compute.data.Page;
3232
import org.elasticsearch.compute.operator.Driver;
@@ -35,6 +35,7 @@
3535
import org.elasticsearch.compute.operator.ResponseHeadersCollector;
3636
import org.elasticsearch.compute.operator.exchange.ExchangeResponse;
3737
import org.elasticsearch.compute.operator.exchange.ExchangeService;
38+
import org.elasticsearch.compute.operator.exchange.ExchangeSink;
3839
import org.elasticsearch.compute.operator.exchange.ExchangeSinkHandler;
3940
import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler;
4041
import org.elasticsearch.core.IOUtils;
@@ -279,7 +280,7 @@ private ActionListener<Void> cancelOnFailure(CancellableTask task, AtomicBoolean
279280
}
280281

281282
void runCompute(CancellableTask task, ComputeContext context, PhysicalPlan plan, ActionListener<List<DriverProfile>> listener) {
282-
listener = ActionListener.runAfter(listener, () -> Releasables.close(context.searchContexts));
283+
listener = ActionListener.runBefore(listener, () -> Releasables.close(context.searchContexts));
283284
final List<Driver> drivers;
284285
try {
285286
LocalExecutionPlanner planner = new LocalExecutionPlanner(
@@ -500,37 +501,93 @@ public void writeTo(StreamOutput out) throws IOException {
500501
// TODO: Use an internal action here
501502
public static final String DATA_ACTION_NAME = EsqlQueryAction.NAME + "/data";
502503

504+
private class DataNodeRequestExecutor {
505+
private final DataNodeRequest request;
506+
private final CancellableTask parentTask;
507+
private final ExchangeSinkHandler exchangeSink;
508+
private final ActionListener<DataNodeResponse> listener;
509+
private final List<DriverProfile> driverProfiles;
510+
private final int maxConcurrentShards;
511+
private final ExchangeSink blockingSink; // block until we have completed on all shards or the coordinator has enough data
512+
513+
DataNodeRequestExecutor(
514+
DataNodeRequest request,
515+
CancellableTask parentTask,
516+
ExchangeSinkHandler exchangeSink,
517+
int maxConcurrentShards,
518+
ActionListener<DataNodeResponse> listener
519+
) {
520+
this.request = request;
521+
this.parentTask = parentTask;
522+
this.exchangeSink = exchangeSink;
523+
this.listener = listener;
524+
this.driverProfiles = request.configuration().profile() ? Collections.synchronizedList(new ArrayList<>()) : List.of();
525+
this.maxConcurrentShards = maxConcurrentShards;
526+
this.blockingSink = exchangeSink.createExchangeSink();
527+
}
528+
529+
void start() {
530+
parentTask.addListener(
531+
() -> exchangeService.finishSinkHandler(request.sessionId(), new TaskCancelledException(parentTask.getReasonCancelled()))
532+
);
533+
runBatch(0);
534+
}
535+
536+
private void runBatch(int startBatchIndex) {
537+
final EsqlConfiguration configuration = request.configuration();
538+
final var sessionId = request.sessionId();
539+
final int endBatchIndex = Math.min(startBatchIndex + maxConcurrentShards, request.shardIds().size());
540+
List<ShardId> shardIds = request.shardIds().subList(startBatchIndex, endBatchIndex);
541+
acquireSearchContexts(shardIds, configuration, request.aliasFilters(), ActionListener.wrap(searchContexts -> {
542+
assert ThreadPool.assertCurrentThreadPool(ESQL_THREAD_POOL_NAME, ESQL_WORKER_THREAD_POOL_NAME);
543+
var computeContext = new ComputeContext(sessionId, searchContexts, configuration, null, exchangeSink);
544+
runCompute(
545+
parentTask,
546+
computeContext,
547+
request.plan(),
548+
ActionListener.wrap(profiles -> onBatchCompleted(endBatchIndex, profiles), this::onFailure)
549+
);
550+
}, this::onFailure));
551+
}
552+
553+
private void onBatchCompleted(int lastBatchIndex, List<DriverProfile> batchProfiles) {
554+
if (request.configuration().profile()) {
555+
driverProfiles.addAll(batchProfiles);
556+
}
557+
if (lastBatchIndex < request.shardIds().size() && exchangeSink.isFinished() == false) {
558+
runBatch(lastBatchIndex);
559+
} else {
560+
blockingSink.finish();
561+
// don't return until all pages are fetched
562+
exchangeSink.addCompletionListener(
563+
ContextPreservingActionListener.wrapPreservingContext(
564+
ActionListener.runBefore(
565+
listener.map(nullValue -> new DataNodeResponse(driverProfiles)),
566+
() -> exchangeService.finishSinkHandler(request.sessionId(), null)
567+
),
568+
transportService.getThreadPool().getThreadContext()
569+
)
570+
);
571+
}
572+
}
573+
574+
private void onFailure(Exception e) {
575+
exchangeService.finishSinkHandler(request.sessionId(), e);
576+
listener.onFailure(e);
577+
}
578+
}
579+
503580
private class DataNodeRequestHandler implements TransportRequestHandler<DataNodeRequest> {
504581
@Override
505582
public void messageReceived(DataNodeRequest request, TransportChannel channel, Task task) {
506-
final var parentTask = (CancellableTask) task;
507-
final var sessionId = request.sessionId();
508-
final var exchangeSink = exchangeService.getSinkHandler(sessionId);
509-
parentTask.addListener(() -> exchangeService.finishSinkHandler(sessionId, new TaskCancelledException("task cancelled")));
510-
final ActionListener<DataNodeResponse> listener = new OwningChannelActionListener<>(channel);
511-
final EsqlConfiguration configuration = request.configuration();
512-
acquireSearchContexts(request.shardIds(), configuration, request.aliasFilters(), ActionListener.wrap(searchContexts -> {
513-
assert ThreadPool.assertCurrentThreadPool(ESQL_THREAD_POOL_NAME);
514-
var computeContext = new ComputeContext(sessionId, searchContexts, configuration, null, exchangeSink);
515-
runCompute(parentTask, computeContext, request.plan(), ActionListener.wrap(driverProfiles -> {
516-
// don't return until all pages are fetched
517-
exchangeSink.addCompletionListener(
518-
ContextPreservingActionListener.wrapPreservingContext(
519-
ActionListener.releaseAfter(
520-
listener.map(nullValue -> new DataNodeResponse(driverProfiles)),
521-
() -> exchangeService.finishSinkHandler(sessionId, null)
522-
),
523-
transportService.getThreadPool().getThreadContext()
524-
)
525-
);
526-
}, e -> {
527-
exchangeService.finishSinkHandler(sessionId, e);
528-
listener.onFailure(e);
529-
}));
530-
}, e -> {
531-
exchangeService.finishSinkHandler(sessionId, e);
532-
listener.onFailure(e);
533-
}));
583+
DataNodeRequestExecutor executor = new DataNodeRequestExecutor(
584+
request,
585+
(CancellableTask) task,
586+
exchangeService.getSinkHandler(request.sessionId()),
587+
request.configuration().pragmas().maxConcurrentShardsPerNode(),
588+
new ChannelActionListener<>(channel)
589+
);
590+
executor.start();
534591
}
535592
}
536593

0 commit comments

Comments
 (0)