Skip to content

Commit 56128ee

Browse files
committed
do can match only once
1 parent c20d53c commit 56128ee

File tree

5 files changed

+233
-84
lines changed

5 files changed

+233
-84
lines changed

server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -131,11 +131,8 @@ public void searchShards(Task task, SearchShardsRequest searchShardsRequest, Act
131131
listener.delegateFailureAndWrap((delegate, searchRequest) -> {
132132
Index[] concreteIndices = resolvedIndices.getConcreteLocalIndices();
133133
final Set<ResolvedExpression> indicesAndAliases = indexNameExpressionResolver.resolveExpressions(
134-
135134
project.metadata(),
136-
137135
searchRequest.indices()
138-
139136
);
140137
final Map<String, AliasFilter> aliasFilters = transportSearchAction.buildIndexAliasFilters(
141138
project,
Lines changed: 126 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,126 @@
1+
/*
2+
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
3+
* or more contributor license agreements. Licensed under the Elastic License
4+
* 2.0; you may not use this file except in compliance with the Elastic License
5+
* 2.0.
6+
*/
7+
8+
package org.elasticsearch.xpack.esql.action;
9+
10+
import org.elasticsearch.action.ActionListener;
11+
import org.elasticsearch.action.ActionRequest;
12+
import org.elasticsearch.action.ActionRequestValidationException;
13+
import org.elasticsearch.action.ActionResponse;
14+
import org.elasticsearch.action.ActionType;
15+
import org.elasticsearch.action.support.ActionFilters;
16+
import org.elasticsearch.action.support.HandledTransportAction;
17+
import org.elasticsearch.cluster.node.DiscoveryNode;
18+
import org.elasticsearch.cluster.project.ProjectResolver;
19+
import org.elasticsearch.cluster.service.ClusterService;
20+
import org.elasticsearch.common.io.stream.StreamInput;
21+
import org.elasticsearch.common.io.stream.StreamOutput;
22+
import org.elasticsearch.common.util.Maps;
23+
import org.elasticsearch.index.shard.ShardId;
24+
import org.elasticsearch.injection.guice.Inject;
25+
import org.elasticsearch.tasks.Task;
26+
import org.elasticsearch.threadpool.ThreadPool;
27+
import org.elasticsearch.transport.TransportService;
28+
29+
import java.io.IOException;
30+
import java.util.List;
31+
import java.util.Map;
32+
import java.util.Set;
33+
34+
public class EsqlResolveNodesAction extends HandledTransportAction<
35+
EsqlResolveNodesAction.ResolveNodesRequest,
36+
EsqlResolveNodesAction.ResolveNodesResponse> {
37+
38+
public static final String NAME = "indices:data/read/esql/resolve_nodes";
39+
public static final ActionType<ResolveNodesResponse> TYPE = new ActionType<>(NAME);
40+
41+
private final ClusterService clusterService;
42+
private final ProjectResolver projectResolver;
43+
44+
@Inject
45+
public EsqlResolveNodesAction(
46+
TransportService transportService,
47+
ActionFilters actionFilters,
48+
ClusterService clusterService,
49+
ProjectResolver projectResolver
50+
) {
51+
super(
52+
NAME,
53+
transportService,
54+
actionFilters,
55+
ResolveNodesRequest::new,
56+
transportService.getThreadPool().executor(ThreadPool.Names.SEARCH_COORDINATION)
57+
);
58+
this.clusterService = clusterService;
59+
this.projectResolver = projectResolver;
60+
}
61+
62+
@Override
63+
protected void doExecute(Task task, ResolveNodesRequest request, ActionListener<ResolveNodesResponse> listener) {
64+
ActionListener.completeWith(listener, () -> {
65+
var project = projectResolver.getProjectState(clusterService.state());
66+
var nodes = Maps.<ShardId, List<DiscoveryNode>>newMapWithExpectedSize(request.shardIds.size());
67+
for (var shardId : request.shardIds) {
68+
nodes.put(
69+
shardId,
70+
project.routingTable()
71+
.shardRoutingTable(shardId)
72+
.allShards()
73+
.map(shard -> project.cluster().nodes().get(shard.currentNodeId()))
74+
.toList()
75+
);
76+
}
77+
return new ResolveNodesResponse(nodes);
78+
});
79+
}
80+
81+
public static class ResolveNodesRequest extends ActionRequest {
82+
83+
private final Set<ShardId> shardIds;
84+
85+
public ResolveNodesRequest(Set<ShardId> shardIds) {
86+
this.shardIds = shardIds;
87+
}
88+
89+
public ResolveNodesRequest(StreamInput in) throws IOException {
90+
this.shardIds = in.readCollectionAsImmutableSet(ShardId::new);
91+
}
92+
93+
@Override
94+
public void writeTo(StreamOutput out) throws IOException {
95+
super.writeTo(out);
96+
out.writeCollection(shardIds, StreamOutput::writeWriteable);
97+
}
98+
99+
@Override
100+
public ActionRequestValidationException validate() {
101+
return null;
102+
}
103+
}
104+
105+
public static class ResolveNodesResponse extends ActionResponse {
106+
107+
private final Map<ShardId, List<DiscoveryNode>> nodes;
108+
109+
public ResolveNodesResponse(Map<ShardId, List<DiscoveryNode>> nodes) {
110+
this.nodes = nodes;
111+
}
112+
113+
public ResolveNodesResponse(StreamInput in) throws IOException {
114+
this.nodes = in.readMap(ShardId::new, inner -> inner.readCollectionAsList(DiscoveryNode::new));
115+
}
116+
117+
public Map<ShardId, List<DiscoveryNode>> nodes() {
118+
return nodes;
119+
}
120+
121+
@Override
122+
public void writeTo(StreamOutput out) throws IOException {
123+
out.writeMap(nodes, StreamOutput::writeWriteable, StreamOutput::writeCollection);
124+
}
125+
}
126+
}

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

Lines changed: 41 additions & 34 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,9 @@
3333
import org.elasticsearch.transport.TransportException;
3434
import org.elasticsearch.transport.TransportRequestOptions;
3535
import org.elasticsearch.transport.TransportService;
36+
import org.elasticsearch.xpack.esql.action.EsqlResolveNodesAction;
37+
import org.elasticsearch.xpack.esql.action.EsqlResolveNodesAction.ResolveNodesRequest;
38+
import org.elasticsearch.xpack.esql.action.EsqlResolveNodesAction.ResolveNodesResponse;
3639
import org.elasticsearch.xpack.esql.action.EsqlSearchShardsAction;
3740

3841
import java.util.ArrayList;
@@ -52,8 +55,8 @@
5255
import java.util.concurrent.atomic.AtomicBoolean;
5356
import java.util.concurrent.atomic.AtomicInteger;
5457
import java.util.concurrent.locks.ReentrantLock;
55-
import java.util.function.Predicate;
5658

59+
import static java.util.HashMap.newHashMap;
5760
import static org.elasticsearch.core.TimeValue.timeValueNanos;
5861

5962
/**
@@ -118,28 +121,22 @@ abstract class DataNodeRequestSender {
118121

119122
final void startComputeOnDataNodes(Set<String> concreteIndices, Runnable runOnTaskFailure, ActionListener<ComputeResponse> listener) {
120123
final long startTimeInNanos = System.nanoTime();
121-
searchShards(
122-
originalIndices.indices(),
123-
shardId -> concreteIndices.contains(shardId.getIndexName()),
124-
ActionListener.wrap(targetShards -> {
125-
try (
126-
var computeListener = new ComputeListener(transportService.getThreadPool(), runOnTaskFailure, listener.map(profiles -> {
127-
return new ComputeResponse(
128-
profiles,
129-
timeValueNanos(System.nanoTime() - startTimeInNanos),
130-
targetShards.totalShards(),
131-
targetShards.totalShards() - shardFailures.size() - skippedShards.get(),
132-
targetShards.skippedShards() + skippedShards.get(),
133-
shardFailures.size(),
134-
selectFailures()
135-
);
136-
}))
137-
) {
138-
pendingShardIds.addAll(order(targetShards));
139-
trySendingRequestsForPendingShards(targetShards, computeListener);
140-
}
141-
}, listener::onFailure)
142-
);
124+
searchShards(concreteIndices, ActionListener.wrap(targetShards -> {
125+
try (var computeListener = new ComputeListener(transportService.getThreadPool(), runOnTaskFailure, listener.map(profiles -> {
126+
return new ComputeResponse(
127+
profiles,
128+
timeValueNanos(System.nanoTime() - startTimeInNanos),
129+
targetShards.totalShards(),
130+
targetShards.totalShards() - shardFailures.size() - skippedShards.get(),
131+
targetShards.skippedShards() + skippedShards.get(),
132+
shardFailures.size(),
133+
selectFailures()
134+
);
135+
}))) {
136+
pendingShardIds.addAll(order(targetShards));
137+
trySendingRequestsForPendingShards(targetShards, computeListener);
138+
}
139+
}, listener::onFailure));
143140
}
144141

145142
private static List<ShardId> order(TargetShards targetShards) {
@@ -256,12 +253,11 @@ void onAfter(List<DriverProfile> profiles) {
256253
concurrentRequests.release();
257254
}
258255

259-
if (pendingRetries.isEmpty() == false && remainingTargetShardSearchAttempts.decrementAndGet() > 0) {
256+
if (pendingRetries.isEmpty() == false && remainingTargetShardSearchAttempts.getAndDecrement() > 0) {
260257
ongoingTargetShardResolutionAttempts.incrementAndGet();
261-
var indices = pendingRetries.stream().map(ShardId::getIndexName).distinct().toArray(String[]::new);
262-
searchShards(indices, pendingRetries::contains, computeListener.acquireAvoid().delegateFailure((l, newSearchShards) -> {
263-
for (var entry : newSearchShards.shards.entrySet()) {
264-
targetShards.shards.get(entry.getKey()).remainingNodes.addAll(entry.getValue().remainingNodes);
258+
resolveShards(pendingRetries, computeListener.acquireAvoid().delegateFailure((l, resolutions) -> {
259+
for (var entry : resolutions.entrySet()) {
260+
targetShards.shards.get(entry.getKey()).remainingNodes.addAll(entry.getValue());
265261
}
266262
ongoingTargetShardResolutionAttempts.decrementAndGet();
267263
trySendingRequestsForPendingShards(targetShards, computeListener);
@@ -356,7 +352,7 @@ private void trackShardLevelFailure(ShardId shardId, boolean fatal, Exception or
356352
}
357353

358354
/**
359-
* Result from {@link #searchShards(String[], Predicate, ActionListener)} where can_match is performed to
355+
* Result from {@link #searchShards(Set, ActionListener)} where can_match is performed to
360356
* determine what shards can be skipped and which target nodes are needed for running the ES|QL query
361357
*
362358
* @param shards List of target shards to perform the ES|QL query on
@@ -446,18 +442,18 @@ private List<NodeRequest> selectNodeRequests(TargetShards targetShards) {
446442
* Ideally, the search_shards API should be called before the field-caps API; however, this can lead
447443
* to a situation where the column structure (i.e., matched data types) differs depending on the query.
448444
*/
449-
void searchShards(String[] indices, Predicate<ShardId> predicate, ActionListener<TargetShards> listener) {
445+
void searchShards(Set<String> concreteIndices, ActionListener<TargetShards> listener) {
450446
ActionListener<SearchShardsResponse> searchShardsListener = listener.map(resp -> {
451-
Map<String, DiscoveryNode> nodes = new HashMap<>();
447+
Map<String, DiscoveryNode> nodes = newHashMap(resp.getNodes().size());
452448
for (DiscoveryNode node : resp.getNodes()) {
453449
nodes.put(node.getId(), node);
454450
}
455451
int totalShards = 0;
456452
int skippedShards = 0;
457-
Map<ShardId, TargetShard> shards = new HashMap<>();
453+
Map<ShardId, TargetShard> shards = newHashMap(resp.getGroups().size());
458454
for (SearchShardsGroup group : resp.getGroups()) {
459455
var shardId = group.shardId();
460-
if (predicate.test(shardId) == false) {
456+
if (concreteIndices.contains(shardId.getIndexName()) == false) {
461457
continue;
462458
}
463459
totalShards++;
@@ -475,7 +471,7 @@ void searchShards(String[] indices, Predicate<ShardId> predicate, ActionListener
475471
return new TargetShards(shards, totalShards, skippedShards);
476472
});
477473
var searchShardsRequest = new SearchShardsRequest(
478-
indices,
474+
originalIndices.indices(),
479475
originalIndices.indicesOptions(),
480476
requestFilter,
481477
null,
@@ -492,4 +488,15 @@ void searchShards(String[] indices, Predicate<ShardId> predicate, ActionListener
492488
new ActionListenerResponseHandler<>(searchShardsListener, SearchShardsResponse::new, esqlExecutor)
493489
);
494490
}
491+
492+
void resolveShards(Set<ShardId> shardIds, ActionListener<Map<ShardId, List<DiscoveryNode>>> listener) {
493+
transportService.sendChildRequest(
494+
transportService.getLocalNode(),
495+
EsqlResolveNodesAction.TYPE.name(),
496+
new ResolveNodesRequest(shardIds),
497+
rootTask,
498+
TransportRequestOptions.EMPTY,
499+
new ActionListenerResponseHandler<>(listener.map(ResolveNodesResponse::nodes), ResolveNodesResponse::new, esqlExecutor)
500+
);
501+
}
495502
}

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

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -58,6 +58,7 @@
5858
import org.elasticsearch.xpack.esql.action.EsqlQueryAction;
5959
import org.elasticsearch.xpack.esql.action.EsqlQueryRequestBuilder;
6060
import org.elasticsearch.xpack.esql.action.EsqlResolveFieldsAction;
61+
import org.elasticsearch.xpack.esql.action.EsqlResolveNodesAction;
6162
import org.elasticsearch.xpack.esql.action.EsqlSearchShardsAction;
6263
import org.elasticsearch.xpack.esql.action.RestEsqlAsyncQueryAction;
6364
import org.elasticsearch.xpack.esql.action.RestEsqlDeleteAsyncResultAction;
@@ -239,6 +240,7 @@ public List<ActionHandler> getActions() {
239240
new ActionHandler(XPackUsageFeatureAction.ESQL, EsqlUsageTransportAction.class),
240241
new ActionHandler(XPackInfoFeatureAction.ESQL, EsqlInfoTransportAction.class),
241242
new ActionHandler(EsqlResolveFieldsAction.TYPE, EsqlResolveFieldsAction.class),
243+
new ActionHandler(EsqlResolveNodesAction.TYPE, EsqlResolveNodesAction.class),
242244
new ActionHandler(EsqlSearchShardsAction.TYPE, EsqlSearchShardsAction.class),
243245
new ActionHandler(EsqlAsyncStopAction.INSTANCE, TransportEsqlAsyncStopAction.class),
244246
new ActionHandler(EsqlListQueriesAction.INSTANCE, TransportEsqlListQueriesAction.class),

0 commit comments

Comments
 (0)