Skip to content

Commit 9782179

Browse files
authored
Restructure ES|QL ComputeService (#120312)
This change divides the ComputeService into three smaller components: - ComputeService, responsible for managing high-level computes - ClusterComputeHandler, responsible for handling cluster-level computes across multiple clusters - DataNodeComputeHandler, responsible for managing data-node-level computes within a single cluster This restructuring simplifies the management of these responsibilities.
1 parent ac687e0 commit 9782179

File tree

4 files changed

+769
-633
lines changed

4 files changed

+769
-633
lines changed
Lines changed: 230 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,230 @@
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.plugin;
9+
10+
import org.elasticsearch.action.ActionListener;
11+
import org.elasticsearch.action.ActionListenerResponseHandler;
12+
import org.elasticsearch.action.OriginalIndices;
13+
import org.elasticsearch.action.support.ChannelActionListener;
14+
import org.elasticsearch.compute.EsqlRefCountingListener;
15+
import org.elasticsearch.compute.operator.exchange.ExchangeService;
16+
import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler;
17+
import org.elasticsearch.core.Releasable;
18+
import org.elasticsearch.tasks.CancellableTask;
19+
import org.elasticsearch.tasks.Task;
20+
import org.elasticsearch.tasks.TaskCancelledException;
21+
import org.elasticsearch.threadpool.ThreadPool;
22+
import org.elasticsearch.transport.RemoteClusterService;
23+
import org.elasticsearch.transport.Transport;
24+
import org.elasticsearch.transport.TransportChannel;
25+
import org.elasticsearch.transport.TransportRequestHandler;
26+
import org.elasticsearch.transport.TransportRequestOptions;
27+
import org.elasticsearch.transport.TransportService;
28+
import org.elasticsearch.xpack.esql.action.EsqlExecutionInfo;
29+
import org.elasticsearch.xpack.esql.plan.physical.ExchangeSinkExec;
30+
import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
31+
import org.elasticsearch.xpack.esql.session.Configuration;
32+
33+
import java.util.ArrayList;
34+
import java.util.Arrays;
35+
import java.util.List;
36+
import java.util.Map;
37+
import java.util.Set;
38+
import java.util.concurrent.Executor;
39+
40+
/**
41+
* Manages computes across multiple clusters by sending {@link ClusterComputeRequest} to remote clusters and executing the computes.
42+
* This handler delegates the execution of computes on data nodes within each remote cluster to {@link DataNodeComputeHandler}.
43+
*/
44+
final class ClusterComputeHandler implements TransportRequestHandler<ClusterComputeRequest> {
45+
private final ComputeService computeService;
46+
private final ExchangeService exchangeService;
47+
private final TransportService transportService;
48+
private final Executor esqlExecutor;
49+
private final DataNodeComputeHandler dataNodeComputeHandler;
50+
51+
ClusterComputeHandler(
52+
ComputeService computeService,
53+
ExchangeService exchangeService,
54+
TransportService transportService,
55+
Executor esqlExecutor,
56+
DataNodeComputeHandler dataNodeComputeHandler
57+
) {
58+
this.computeService = computeService;
59+
this.exchangeService = exchangeService;
60+
this.esqlExecutor = esqlExecutor;
61+
this.transportService = transportService;
62+
this.dataNodeComputeHandler = dataNodeComputeHandler;
63+
transportService.registerRequestHandler(ComputeService.CLUSTER_ACTION_NAME, esqlExecutor, ClusterComputeRequest::new, this);
64+
}
65+
66+
void startComputeOnRemoteClusters(
67+
String sessionId,
68+
CancellableTask rootTask,
69+
Configuration configuration,
70+
PhysicalPlan plan,
71+
ExchangeSourceHandler exchangeSource,
72+
List<RemoteCluster> clusters,
73+
ComputeListener computeListener
74+
) {
75+
var queryPragmas = configuration.pragmas();
76+
var linkExchangeListeners = ActionListener.releaseAfter(computeListener.acquireAvoid(), exchangeSource.addEmptySink());
77+
try (EsqlRefCountingListener refs = new EsqlRefCountingListener(linkExchangeListeners)) {
78+
for (RemoteCluster cluster : clusters) {
79+
final var childSessionId = computeService.newChildSession(sessionId);
80+
ExchangeService.openExchange(
81+
transportService,
82+
cluster.connection,
83+
childSessionId,
84+
queryPragmas.exchangeBufferSize(),
85+
esqlExecutor,
86+
refs.acquire().delegateFailureAndWrap((l, unused) -> {
87+
var remoteSink = exchangeService.newRemoteSink(rootTask, childSessionId, transportService, cluster.connection);
88+
exchangeSource.addRemoteSink(remoteSink, true, queryPragmas.concurrentExchangeClients(), ActionListener.noop());
89+
var remotePlan = new RemoteClusterPlan(plan, cluster.concreteIndices, cluster.originalIndices);
90+
var clusterRequest = new ClusterComputeRequest(cluster.clusterAlias, childSessionId, configuration, remotePlan);
91+
var clusterListener = ActionListener.runBefore(
92+
computeListener.acquireCompute(cluster.clusterAlias()),
93+
() -> l.onResponse(null)
94+
);
95+
transportService.sendChildRequest(
96+
cluster.connection,
97+
ComputeService.CLUSTER_ACTION_NAME,
98+
clusterRequest,
99+
rootTask,
100+
TransportRequestOptions.EMPTY,
101+
new ActionListenerResponseHandler<>(clusterListener, ComputeResponse::new, esqlExecutor)
102+
);
103+
})
104+
);
105+
}
106+
}
107+
}
108+
109+
List<RemoteCluster> getRemoteClusters(
110+
Map<String, OriginalIndices> clusterToConcreteIndices,
111+
Map<String, OriginalIndices> clusterToOriginalIndices
112+
) {
113+
List<RemoteCluster> remoteClusters = new ArrayList<>(clusterToConcreteIndices.size());
114+
RemoteClusterService remoteClusterService = transportService.getRemoteClusterService();
115+
for (Map.Entry<String, OriginalIndices> e : clusterToConcreteIndices.entrySet()) {
116+
String clusterAlias = e.getKey();
117+
OriginalIndices concreteIndices = clusterToConcreteIndices.get(clusterAlias);
118+
OriginalIndices originalIndices = clusterToOriginalIndices.get(clusterAlias);
119+
if (originalIndices == null) {
120+
assert false : "can't find original indices for cluster " + clusterAlias;
121+
throw new IllegalStateException("can't find original indices for cluster " + clusterAlias);
122+
}
123+
if (concreteIndices.indices().length > 0) {
124+
Transport.Connection connection = remoteClusterService.getConnection(clusterAlias);
125+
remoteClusters.add(new RemoteCluster(clusterAlias, connection, concreteIndices.indices(), originalIndices));
126+
}
127+
}
128+
return remoteClusters;
129+
}
130+
131+
record RemoteCluster(String clusterAlias, Transport.Connection connection, String[] concreteIndices, OriginalIndices originalIndices) {
132+
133+
}
134+
135+
@Override
136+
public void messageReceived(ClusterComputeRequest request, TransportChannel channel, Task task) {
137+
ChannelActionListener<ComputeResponse> listener = new ChannelActionListener<>(channel);
138+
RemoteClusterPlan remoteClusterPlan = request.remoteClusterPlan();
139+
var plan = remoteClusterPlan.plan();
140+
if (plan instanceof ExchangeSinkExec == false) {
141+
listener.onFailure(new IllegalStateException("expected exchange sink for a remote compute; got " + plan));
142+
return;
143+
}
144+
String clusterAlias = request.clusterAlias();
145+
/*
146+
* This handler runs only on remote cluster coordinators, so it creates a new local EsqlExecutionInfo object to record
147+
* execution metadata for ES|QL processing local to this cluster. The execution info will be copied into the
148+
* ComputeResponse that is sent back to the primary coordinating cluster.
149+
*/
150+
EsqlExecutionInfo execInfo = new EsqlExecutionInfo(true);
151+
execInfo.swapCluster(clusterAlias, (k, v) -> new EsqlExecutionInfo.Cluster(clusterAlias, Arrays.toString(request.indices())));
152+
CancellableTask cancellable = (CancellableTask) task;
153+
try (var computeListener = ComputeListener.create(clusterAlias, transportService, cancellable, execInfo, listener)) {
154+
runComputeOnRemoteCluster(
155+
clusterAlias,
156+
request.sessionId(),
157+
(CancellableTask) task,
158+
request.configuration(),
159+
(ExchangeSinkExec) plan,
160+
Set.of(remoteClusterPlan.targetIndices()),
161+
remoteClusterPlan.originalIndices(),
162+
execInfo,
163+
computeListener
164+
);
165+
}
166+
}
167+
168+
/**
169+
* Performs a compute on a remote cluster. The output pages are placed in an exchange sink specified by
170+
* {@code globalSessionId}. The coordinator on the main cluster will poll pages from there.
171+
* <p>
172+
* Currently, the coordinator on the remote cluster polls pages from data nodes within the remote cluster
173+
* and performs cluster-level reduction before sending pages to the querying cluster. This reduction aims
174+
* to minimize data transfers across clusters but may require additional CPU resources for operations like
175+
* aggregations.
176+
*/
177+
void runComputeOnRemoteCluster(
178+
String clusterAlias,
179+
String globalSessionId,
180+
CancellableTask parentTask,
181+
Configuration configuration,
182+
ExchangeSinkExec plan,
183+
Set<String> concreteIndices,
184+
OriginalIndices originalIndices,
185+
EsqlExecutionInfo executionInfo,
186+
ComputeListener computeListener
187+
) {
188+
final var exchangeSink = exchangeService.getSinkHandler(globalSessionId);
189+
parentTask.addListener(
190+
() -> exchangeService.finishSinkHandler(globalSessionId, new TaskCancelledException(parentTask.getReasonCancelled()))
191+
);
192+
final String localSessionId = clusterAlias + ":" + globalSessionId;
193+
final PhysicalPlan coordinatorPlan = ComputeService.reductionPlan(plan, true);
194+
var exchangeSource = new ExchangeSourceHandler(
195+
configuration.pragmas().exchangeBufferSize(),
196+
transportService.getThreadPool().executor(ThreadPool.Names.SEARCH),
197+
computeListener.acquireAvoid()
198+
);
199+
try (Releasable ignored = exchangeSource.addEmptySink()) {
200+
exchangeSink.addCompletionListener(computeListener.acquireAvoid());
201+
computeService.runCompute(
202+
parentTask,
203+
new ComputeContext(
204+
localSessionId,
205+
clusterAlias,
206+
List.of(),
207+
configuration,
208+
configuration.newFoldContext(),
209+
exchangeSource,
210+
exchangeSink
211+
),
212+
coordinatorPlan,
213+
computeListener.acquireCompute(clusterAlias)
214+
);
215+
dataNodeComputeHandler.startComputeOnDataNodes(
216+
localSessionId,
217+
clusterAlias,
218+
parentTask,
219+
configuration,
220+
plan,
221+
concreteIndices,
222+
originalIndices,
223+
exchangeSource,
224+
executionInfo,
225+
computeListener
226+
);
227+
}
228+
}
229+
230+
}
Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,31 @@
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.plugin;
9+
10+
import org.elasticsearch.compute.operator.exchange.ExchangeSinkHandler;
11+
import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler;
12+
import org.elasticsearch.index.query.SearchExecutionContext;
13+
import org.elasticsearch.search.internal.SearchContext;
14+
import org.elasticsearch.xpack.esql.core.expression.FoldContext;
15+
import org.elasticsearch.xpack.esql.session.Configuration;
16+
17+
import java.util.List;
18+
19+
record ComputeContext(
20+
String sessionId,
21+
String clusterAlias,
22+
List<SearchContext> searchContexts,
23+
Configuration configuration,
24+
FoldContext foldCtx,
25+
ExchangeSourceHandler exchangeSource,
26+
ExchangeSinkHandler exchangeSink
27+
) {
28+
List<SearchExecutionContext> searchExecutionContexts() {
29+
return searchContexts.stream().map(SearchContext::getSearchExecutionContext).toList();
30+
}
31+
}

0 commit comments

Comments
 (0)